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

[01/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-313 bc18635a9 -> 401835cc6


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java
new file mode 100644
index 0000000..5b1b6a8
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.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.collections;
+
+import com.google.common.collect.*;
+import org.apache.hadoop.io.*;
+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.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class HadoopHashMapSelfTest extends HadoopAbstractMapTest {
+
+    public void _testAllocation() throws Exception {
+        final GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+        long size = 3L * 1024 * 1024 * 1024;
+
+        final long chunk = 16;// * 1024;
+
+        final int page = 4 * 1024;
+
+        final int writes = chunk < page ? 1 : (int)(chunk / page);
+
+        final long cnt = size / chunk;
+
+        assert cnt < Integer.MAX_VALUE;
+
+        final int threads = 4;
+
+        long start = System.currentTimeMillis();
+
+        multithreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int cnt0 = (int)(cnt / threads);
+
+                for (int i = 0; i < cnt0; i++) {
+                    long ptr = mem.allocate(chunk);
+
+                    for (int j = 0; j < writes; j++)
+                        mem.writeInt(ptr + j * page, 100500);
+                }
+
+                return null;
+            }
+        }, threads);
+
+        X.println("End: " + (System.currentTimeMillis() - start) + " mem: " + mem.allocatedSize() + " cnt: " + cnt);
+
+        Thread.sleep(30000);
+    }
+
+
+    /** */
+    public void testMapSimple() throws Exception {
+        GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+//        mem.listen(new GridOffHeapEventListener() {
+//            @Override public void onEvent(GridOffHeapEvent evt) {
+//                if (evt == GridOffHeapEvent.ALLOCATE)
+//                    U.dumpStack();
+//            }
+//        });
+
+        Random rnd = new Random();
+
+        int mapSize = 16 << rnd.nextInt(3);
+
+        HadoopTaskContext taskCtx = new TaskContext();
+
+        final HadoopHashMultimap m = new HadoopHashMultimap(new JobInfo(), mem, mapSize);
+
+        HadoopMultimap.Adder a = m.startAdding(taskCtx);
+
+        Multimap<Integer, Integer> mm = ArrayListMultimap.create();
+
+        for (int i = 0, vals = 4 * mapSize + rnd.nextInt(25); i < vals; i++) {
+            int key = rnd.nextInt(mapSize);
+            int val = rnd.nextInt();
+
+            a.write(new IntWritable(key), new IntWritable(val));
+            mm.put(key, val);
+
+            X.println("k: " + key + " v: " + val);
+
+            a.close();
+
+            check(m, mm, taskCtx);
+
+            a = m.startAdding(taskCtx);
+        }
+
+//        a.add(new IntWritable(10), new IntWritable(2));
+//        mm.put(10, 2);
+//        check(m, mm);
+
+        a.close();
+
+        X.println("Alloc: " + mem.allocatedSize());
+
+        m.close();
+
+        assertEquals(0, mem.allocatedSize());
+    }
+
+    private void check(HadoopHashMultimap m, Multimap<Integer, Integer> mm, HadoopTaskContext taskCtx) throws Exception {
+        final HadoopTaskInput in = m.input(taskCtx);
+
+        Map<Integer, Collection<Integer>> mmm = mm.asMap();
+
+        int keys = 0;
+
+        while (in.next()) {
+            keys++;
+
+            IntWritable k = (IntWritable)in.key();
+
+            assertNotNull(k);
+
+            ArrayList<Integer> vs = new ArrayList<>();
+
+            Iterator<?> it = in.values();
+
+            while (it.hasNext())
+                vs.add(((IntWritable) it.next()).get());
+
+            Collection<Integer> exp = mmm.get(k.get());
+
+            assertEquals(sorted(exp), sorted(vs));
+        }
+
+        X.println("keys: " + keys + " cap: " + m.capacity());
+
+        assertEquals(mmm.size(), keys);
+
+        assertEquals(m.keys(), keys);
+
+        in.close();
+    }
+
+    private GridLongList sorted(Collection<Integer> col) {
+        GridLongList lst = new GridLongList(col.size());
+
+        for (Integer i : col)
+            lst.add(i);
+
+        return lst.sort();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
new file mode 100644
index 0000000..8a046e0
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.google.common.collect.*;
+import org.apache.hadoop.io.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.lang.Math.*;
+import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
+
+/**
+ * Skip list tests.
+ */
+public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
+    /**
+     *
+     */
+    public void testLevel() {
+        Random rnd = new GridRandom();
+
+        int[] levelsCnts = new int[32];
+
+        int all = 10000;
+
+        for (int i = 0; i < all; i++) {
+            int level = HadoopSkipList.randomLevel(rnd);
+
+            levelsCnts[level]++;
+        }
+
+        X.println("Distribution: " + Arrays.toString(levelsCnts));
+
+        for (int level = 0; level < levelsCnts.length; level++) {
+            int exp = (level + 1) == levelsCnts.length ? 0 : all >>> (level + 1);
+
+            double precission = 0.72 / Math.max(32 >>> level, 1);
+
+            int sigma = max((int)ceil(precission * exp), 5);
+
+            X.println("Level: " + level + " exp: " + exp + " act: " + levelsCnts[level] + " precission: " + precission +
+                " sigma: " + sigma);
+
+            assertTrue(abs(exp - levelsCnts[level]) <= sigma);
+        }
+    }
+
+    public void testMapSimple() throws Exception {
+        GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+//        mem.listen(new GridOffHeapEventListener() {
+//            @Override public void onEvent(GridOffHeapEvent evt) {
+//                if (evt == GridOffHeapEvent.ALLOCATE)
+//                    U.dumpStack();
+//            }
+//        });
+
+        Random rnd = new Random();
+
+        int mapSize = 16 << rnd.nextInt(6);
+
+        HadoopJobInfo job = new JobInfo();
+
+        HadoopTaskContext taskCtx = new TaskContext();
+
+        HadoopMultimap m = new HadoopSkipList(job, mem);
+
+        HadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx);
+
+        Multimap<Integer, Integer> mm = ArrayListMultimap.create();
+        Multimap<Integer, Integer> vis = ArrayListMultimap.create();
+
+        for (int i = 0, vals = 4 * mapSize + rnd.nextInt(25); i < vals; i++) {
+            int key = rnd.nextInt(mapSize);
+            int val = rnd.nextInt();
+
+            a.write(new IntWritable(key), new IntWritable(val));
+            mm.put(key, val);
+
+            X.println("k: " + key + " v: " + val);
+
+            a.close();
+
+            check(m, mm, vis, taskCtx);
+
+            a = m.startAdding(taskCtx);
+        }
+
+//        a.add(new IntWritable(10), new IntWritable(2));
+//        mm.put(10, 2);
+//        check(m, mm);
+
+        a.close();
+
+        X.println("Alloc: " + mem.allocatedSize());
+
+        m.close();
+
+        assertEquals(0, mem.allocatedSize());
+    }
+
+    private void check(HadoopMultimap m, Multimap<Integer, Integer> mm, final Multimap<Integer, Integer> vis, HadoopTaskContext taskCtx)
+        throws Exception {
+        final HadoopTaskInput in = m.input(taskCtx);
+
+        Map<Integer, Collection<Integer>> mmm = mm.asMap();
+
+        int keys = 0;
+
+        int prevKey = Integer.MIN_VALUE;
+
+        while (in.next()) {
+            keys++;
+
+            IntWritable k = (IntWritable)in.key();
+
+            assertNotNull(k);
+
+            assertTrue(k.get() > prevKey);
+
+            prevKey = k.get();
+
+            Deque<Integer> vs = new LinkedList<>();
+
+            Iterator<?> it = in.values();
+
+            while (it.hasNext())
+                vs.addFirst(((IntWritable) it.next()).get());
+
+            Collection<Integer> exp = mmm.get(k.get());
+
+            assertEquals(exp, vs);
+        }
+
+        assertEquals(mmm.size(), keys);
+
+//!        assertEquals(m.keys(), keys);
+
+        // Check visitor.
+
+        final byte[] buf = new byte[4];
+
+        final GridDataInput dataInput = new GridUnsafeDataInput();
+
+        m.visit(false, new HadoopConcurrentHashMultimap.Visitor() {
+            /** */
+            IntWritable key = new IntWritable();
+
+            /** */
+            IntWritable val = new IntWritable();
+
+            @Override public void onKey(long keyPtr, int keySize) {
+                read(keyPtr, keySize, key);
+            }
+
+            @Override public void onValue(long valPtr, int valSize) {
+                read(valPtr, valSize, val);
+
+                vis.put(key.get(), val.get());
+            }
+
+            private void read(long ptr, int size, Writable w) {
+                assert size == 4 : size;
+
+                UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF, size);
+
+                dataInput.bytes(buf, size);
+
+                try {
+                    w.readFields(dataInput);
+                }
+                catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        });
+
+//        X.println("vis: " + vis);
+
+        assertEquals(mm, vis);
+
+        in.close();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testMultiThreaded() throws Exception {
+        GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+        X.println("___ Started");
+
+        Random rnd = new GridRandom();
+
+        for (int i = 0; i < 20; i++) {
+            HadoopJobInfo job = new JobInfo();
+
+            final HadoopTaskContext taskCtx = new TaskContext();
+
+            final HadoopMultimap m = new HadoopSkipList(job, mem);
+
+            final ConcurrentMap<Integer, Collection<Integer>> mm = new ConcurrentHashMap<>();
+
+            X.println("___ MT");
+
+            multithreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    X.println("___ TH in");
+
+                    Random rnd = new GridRandom();
+
+                    IntWritable key = new IntWritable();
+                    IntWritable val = new IntWritable();
+
+                    HadoopMultimap.Adder a = m.startAdding(taskCtx);
+
+                    for (int i = 0; i < 50000; i++) {
+                        int k = rnd.nextInt(32000);
+                        int v = rnd.nextInt();
+
+                        key.set(k);
+                        val.set(v);
+
+                        a.write(key, val);
+
+                        Collection<Integer> list = mm.get(k);
+
+                        if (list == null) {
+                            list = new ConcurrentLinkedQueue<>();
+
+                            Collection<Integer> old = mm.putIfAbsent(k, list);
+
+                            if (old != null)
+                                list = old;
+                        }
+
+                        list.add(v);
+                    }
+
+                    a.close();
+
+                    X.println("___ TH out");
+
+                    return null;
+                }
+            }, 3 + rnd.nextInt(27));
+
+            HadoopTaskInput in = m.input(taskCtx);
+
+            int prevKey = Integer.MIN_VALUE;
+
+            while (in.next()) {
+                IntWritable key = (IntWritable)in.key();
+
+                assertTrue(key.get() > prevKey);
+
+                prevKey = key.get();
+
+                Iterator<?> valsIter = in.values();
+
+                Collection<Integer> vals = mm.remove(key.get());
+
+                assertNotNull(vals);
+
+                while (valsIter.hasNext()) {
+                    IntWritable val = (IntWritable) valsIter.next();
+
+                    assertTrue(vals.remove(val.get()));
+                }
+
+                assertTrue(vals.isEmpty());
+            }
+
+            in.close();
+            m.close();
+
+            assertEquals(0, mem.allocatedSize());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 39a537b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java
+++ /dev/null
@@ -1,151 +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 org.apache.ignite.testframework.junits.common.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- *
- */
-public class GridHadoopDataStreamSelfTest extends GridCommonAbstractTest {
-
-    public void testStreams() throws IOException {
-        GridUnsafeMemory mem = new GridUnsafeMemory(0);
-
-        GridHadoopDataOutStream out = new GridHadoopDataOutStream(mem);
-
-        int size = 4 * 1024;
-
-        final long ptr = mem.allocate(size);
-
-        out.buffer().set(ptr, size);
-
-        out.writeBoolean(false);
-        out.writeBoolean(true);
-        out.writeBoolean(false);
-        out.write(17);
-        out.write(121);
-        out.write(0xfafa);
-        out.writeByte(17);
-        out.writeByte(121);
-        out.writeByte(0xfafa);
-        out.writeChar('z');
-        out.writeChar('o');
-        out.writeChar('r');
-        out.writeShort(100);
-        out.writeShort(Short.MIN_VALUE);
-        out.writeShort(Short.MAX_VALUE);
-        out.writeShort(65535);
-        out.writeShort(65536); // 0
-        out.writeInt(Integer.MAX_VALUE);
-        out.writeInt(Integer.MIN_VALUE);
-        out.writeInt(-1);
-        out.writeInt(0);
-        out.writeInt(1);
-        out.writeFloat(0.33f);
-        out.writeFloat(0.5f);
-        out.writeFloat(-0.7f);
-        out.writeFloat(Float.MAX_VALUE);
-        out.writeFloat(Float.MIN_VALUE);
-        out.writeFloat(Float.MIN_NORMAL);
-        out.writeFloat(Float.POSITIVE_INFINITY);
-        out.writeFloat(Float.NEGATIVE_INFINITY);
-        out.writeFloat(Float.NaN);
-        out.writeDouble(-12312312.3333333336666779);
-        out.writeDouble(123123.234);
-        out.writeDouble(Double.MAX_VALUE);
-        out.writeDouble(Double.MIN_VALUE);
-        out.writeDouble(Double.MIN_NORMAL);
-        out.writeDouble(Double.NEGATIVE_INFINITY);
-        out.writeDouble(Double.POSITIVE_INFINITY);
-        out.writeDouble(Double.NaN);
-        out.writeLong(Long.MAX_VALUE);
-        out.writeLong(Long.MIN_VALUE);
-        out.writeLong(0);
-        out.writeLong(-1L);
-        out.write(new byte[]{1,2,3});
-        out.write(new byte[]{0,1,2,3}, 1, 2);
-        out.writeUTF("mom washes rum");
-
-        GridHadoopDataInStream in = new GridHadoopDataInStream(mem);
-
-        in.buffer().set(ptr, out.buffer().pointer());
-
-        assertEquals(false, in.readBoolean());
-        assertEquals(true, in.readBoolean());
-        assertEquals(false, in.readBoolean());
-        assertEquals(17, in.read());
-        assertEquals(121, in.read());
-        assertEquals(0xfa, in.read());
-        assertEquals(17, in.readByte());
-        assertEquals(121, in.readByte());
-        assertEquals((byte)0xfa, in.readByte());
-        assertEquals('z', in.readChar());
-        assertEquals('o', in.readChar());
-        assertEquals('r', in.readChar());
-        assertEquals(100, in.readShort());
-        assertEquals(Short.MIN_VALUE, in.readShort());
-        assertEquals(Short.MAX_VALUE, in.readShort());
-        assertEquals(-1, in.readShort());
-        assertEquals(0, in.readShort());
-        assertEquals(Integer.MAX_VALUE, in.readInt());
-        assertEquals(Integer.MIN_VALUE, in.readInt());
-        assertEquals(-1, in.readInt());
-        assertEquals(0, in.readInt());
-        assertEquals(1, in.readInt());
-        assertEquals(0.33f, in.readFloat());
-        assertEquals(0.5f, in.readFloat());
-        assertEquals(-0.7f, in.readFloat());
-        assertEquals(Float.MAX_VALUE, in.readFloat());
-        assertEquals(Float.MIN_VALUE, in.readFloat());
-        assertEquals(Float.MIN_NORMAL, in.readFloat());
-        assertEquals(Float.POSITIVE_INFINITY, in.readFloat());
-        assertEquals(Float.NEGATIVE_INFINITY, in.readFloat());
-        assertEquals(Float.NaN, in.readFloat());
-        assertEquals(-12312312.3333333336666779, in.readDouble());
-        assertEquals(123123.234, in.readDouble());
-        assertEquals(Double.MAX_VALUE, in.readDouble());
-        assertEquals(Double.MIN_VALUE, in.readDouble());
-        assertEquals(Double.MIN_NORMAL, in.readDouble());
-        assertEquals(Double.NEGATIVE_INFINITY, in.readDouble());
-        assertEquals(Double.POSITIVE_INFINITY, in.readDouble());
-        assertEquals(Double.NaN, in.readDouble());
-        assertEquals(Long.MAX_VALUE, in.readLong());
-        assertEquals(Long.MIN_VALUE, in.readLong());
-        assertEquals(0, in.readLong());
-        assertEquals(-1, in.readLong());
-
-        byte[] b = new byte[3];
-
-        in.read(b);
-
-        assertTrue(Arrays.equals(new byte[]{1,2,3}, b));
-
-        b = new byte[4];
-
-        in.read(b, 1, 2);
-
-        assertTrue(Arrays.equals(new byte[]{0, 1, 2, 0}, b));
-
-        assertEquals("mom washes rum", in.readUTF());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java
new file mode 100644
index 0000000..48b99ab
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class HadoopDataStreamSelfTest extends GridCommonAbstractTest {
+
+    public void testStreams() throws IOException {
+        GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+        HadoopDataOutStream out = new HadoopDataOutStream(mem);
+
+        int size = 4 * 1024;
+
+        final long ptr = mem.allocate(size);
+
+        out.buffer().set(ptr, size);
+
+        out.writeBoolean(false);
+        out.writeBoolean(true);
+        out.writeBoolean(false);
+        out.write(17);
+        out.write(121);
+        out.write(0xfafa);
+        out.writeByte(17);
+        out.writeByte(121);
+        out.writeByte(0xfafa);
+        out.writeChar('z');
+        out.writeChar('o');
+        out.writeChar('r');
+        out.writeShort(100);
+        out.writeShort(Short.MIN_VALUE);
+        out.writeShort(Short.MAX_VALUE);
+        out.writeShort(65535);
+        out.writeShort(65536); // 0
+        out.writeInt(Integer.MAX_VALUE);
+        out.writeInt(Integer.MIN_VALUE);
+        out.writeInt(-1);
+        out.writeInt(0);
+        out.writeInt(1);
+        out.writeFloat(0.33f);
+        out.writeFloat(0.5f);
+        out.writeFloat(-0.7f);
+        out.writeFloat(Float.MAX_VALUE);
+        out.writeFloat(Float.MIN_VALUE);
+        out.writeFloat(Float.MIN_NORMAL);
+        out.writeFloat(Float.POSITIVE_INFINITY);
+        out.writeFloat(Float.NEGATIVE_INFINITY);
+        out.writeFloat(Float.NaN);
+        out.writeDouble(-12312312.3333333336666779);
+        out.writeDouble(123123.234);
+        out.writeDouble(Double.MAX_VALUE);
+        out.writeDouble(Double.MIN_VALUE);
+        out.writeDouble(Double.MIN_NORMAL);
+        out.writeDouble(Double.NEGATIVE_INFINITY);
+        out.writeDouble(Double.POSITIVE_INFINITY);
+        out.writeDouble(Double.NaN);
+        out.writeLong(Long.MAX_VALUE);
+        out.writeLong(Long.MIN_VALUE);
+        out.writeLong(0);
+        out.writeLong(-1L);
+        out.write(new byte[]{1,2,3});
+        out.write(new byte[]{0,1,2,3}, 1, 2);
+        out.writeUTF("mom washes rum");
+
+        HadoopDataInStream in = new HadoopDataInStream(mem);
+
+        in.buffer().set(ptr, out.buffer().pointer());
+
+        assertEquals(false, in.readBoolean());
+        assertEquals(true, in.readBoolean());
+        assertEquals(false, in.readBoolean());
+        assertEquals(17, in.read());
+        assertEquals(121, in.read());
+        assertEquals(0xfa, in.read());
+        assertEquals(17, in.readByte());
+        assertEquals(121, in.readByte());
+        assertEquals((byte)0xfa, in.readByte());
+        assertEquals('z', in.readChar());
+        assertEquals('o', in.readChar());
+        assertEquals('r', in.readChar());
+        assertEquals(100, in.readShort());
+        assertEquals(Short.MIN_VALUE, in.readShort());
+        assertEquals(Short.MAX_VALUE, in.readShort());
+        assertEquals(-1, in.readShort());
+        assertEquals(0, in.readShort());
+        assertEquals(Integer.MAX_VALUE, in.readInt());
+        assertEquals(Integer.MIN_VALUE, in.readInt());
+        assertEquals(-1, in.readInt());
+        assertEquals(0, in.readInt());
+        assertEquals(1, in.readInt());
+        assertEquals(0.33f, in.readFloat());
+        assertEquals(0.5f, in.readFloat());
+        assertEquals(-0.7f, in.readFloat());
+        assertEquals(Float.MAX_VALUE, in.readFloat());
+        assertEquals(Float.MIN_VALUE, in.readFloat());
+        assertEquals(Float.MIN_NORMAL, in.readFloat());
+        assertEquals(Float.POSITIVE_INFINITY, in.readFloat());
+        assertEquals(Float.NEGATIVE_INFINITY, in.readFloat());
+        assertEquals(Float.NaN, in.readFloat());
+        assertEquals(-12312312.3333333336666779, in.readDouble());
+        assertEquals(123123.234, in.readDouble());
+        assertEquals(Double.MAX_VALUE, in.readDouble());
+        assertEquals(Double.MIN_VALUE, in.readDouble());
+        assertEquals(Double.MIN_NORMAL, in.readDouble());
+        assertEquals(Double.NEGATIVE_INFINITY, in.readDouble());
+        assertEquals(Double.POSITIVE_INFINITY, in.readDouble());
+        assertEquals(Double.NaN, in.readDouble());
+        assertEquals(Long.MAX_VALUE, in.readLong());
+        assertEquals(Long.MIN_VALUE, in.readLong());
+        assertEquals(0, in.readLong());
+        assertEquals(-1, in.readLong());
+
+        byte[] b = new byte[3];
+
+        in.read(b);
+
+        assertTrue(Arrays.equals(new byte[]{1,2,3}, b));
+
+        b = new byte[4];
+
+        in.read(b, 1, 2);
+
+        assertTrue(Arrays.equals(new byte[]{0, 1, 2, 0}, b));
+
+        assertEquals("mom washes rum", in.readUTF());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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
deleted file mode 100644
index 52bfa98..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.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.processors.hadoop.taskexecutor.external;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Job tracker self test.
- */
-public class GridHadoopExternalTaskExecutionSelfTest extends GridHadoopAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(true);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleTaskSubmit() throws Exception {
-        String testInputFile = "/test";
-
-        prepareTestFile(testInputFile);
-
-        Configuration cfg = new Configuration();
-
-        setupFileSystems(cfg);
-
-        Job job = Job.getInstance(cfg);
-
-        job.setMapperClass(TestMapper.class);
-        job.setCombinerClass(TestReducer.class);
-        job.setReducerClass(TestReducer.class);
-
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(IntWritable.class);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setNumReduceTasks(1);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/" + testInputFile));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
-
-        job.setJarByClass(getClass());
-
-        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
-            createJobInfo(job.getConfiguration()));
-
-        fut.get();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapperException() throws Exception {
-        String testInputFile = "/test";
-
-        prepareTestFile(testInputFile);
-
-        Configuration cfg = new Configuration();
-
-        setupFileSystems(cfg);
-
-        Job job = Job.getInstance(cfg);
-
-        job.setMapperClass(TestFailingMapper.class);
-        job.setCombinerClass(TestReducer.class);
-        job.setReducerClass(TestReducer.class);
-
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(IntWritable.class);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setNumReduceTasks(1);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/" + testInputFile));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
-
-        job.setJarByClass(getClass());
-
-        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
-            createJobInfo(job.getConfiguration()));
-
-        try {
-            fut.get();
-        }
-        catch (IgniteCheckedException e) {
-            IOException exp = X.cause(e, IOException.class);
-
-            assertNotNull(exp);
-            assertEquals("Test failure", exp.getMessage());
-        }
-    }
-
-    /**
-     * @param filePath File path to prepare.
-     * @throws Exception If failed.
-     */
-    private void prepareTestFile(String filePath) throws Exception {
-        IgniteFs igfs = grid(0).fileSystem(igfsName);
-
-        try (IgfsOutputStream out = igfs.create(new IgfsPath(filePath), true)) {
-            PrintWriter wr = new PrintWriter(new OutputStreamWriter(out));
-
-            for (int i = 0; i < 1000; i++)
-                wr.println("Hello, world: " + i);
-
-            wr.flush();
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
-        /** One constant. */
-        private IntWritable one = new IntWritable(1);
-
-        /** Line constant. */
-        private Text line = new Text("line");
-
-        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            ctx.write(line, one);
-        }
-    }
-
-    /**
-     * Failing mapper.
-     */
-    private static class TestFailingMapper extends Mapper<Object, Text, Text, IntWritable> {
-        @Override protected void map(Object key, Text val, Context c) throws IOException, InterruptedException {
-            throw new IOException("Test failure");
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
-        /** Line constant. */
-        private Text line = new Text("line");
-
-        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
-            super.setup(ctx);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx)
-            throws IOException, InterruptedException {
-            int s = 0;
-
-            for (IntWritable val : values)
-                s += val.get();
-
-            System.out.println(">>>> Reduced: " + s);
-
-            ctx.write(line, new IntWritable(s));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
new file mode 100644
index 0000000..59ac445
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Job tracker self test.
+ */
+public class HadoopExternalTaskExecutionSelfTest extends HadoopAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(true);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleTaskSubmit() throws Exception {
+        String testInputFile = "/test";
+
+        prepareTestFile(testInputFile);
+
+        Configuration cfg = new Configuration();
+
+        setupFileSystems(cfg);
+
+        Job job = Job.getInstance(cfg);
+
+        job.setMapperClass(TestMapper.class);
+        job.setCombinerClass(TestReducer.class);
+        job.setReducerClass(TestReducer.class);
+
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(IntWritable.class);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setNumReduceTasks(1);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/" + testInputFile));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
+
+        job.setJarByClass(getClass());
+
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 1),
+            createJobInfo(job.getConfiguration()));
+
+        fut.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapperException() throws Exception {
+        String testInputFile = "/test";
+
+        prepareTestFile(testInputFile);
+
+        Configuration cfg = new Configuration();
+
+        setupFileSystems(cfg);
+
+        Job job = Job.getInstance(cfg);
+
+        job.setMapperClass(TestFailingMapper.class);
+        job.setCombinerClass(TestReducer.class);
+        job.setReducerClass(TestReducer.class);
+
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(IntWritable.class);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setNumReduceTasks(1);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/" + testInputFile));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
+
+        job.setJarByClass(getClass());
+
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 1),
+            createJobInfo(job.getConfiguration()));
+
+        try {
+            fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            IOException exp = X.cause(e, IOException.class);
+
+            assertNotNull(exp);
+            assertEquals("Test failure", exp.getMessage());
+        }
+    }
+
+    /**
+     * @param filePath File path to prepare.
+     * @throws Exception If failed.
+     */
+    private void prepareTestFile(String filePath) throws Exception {
+        IgniteFileSystem igfs = grid(0).fileSystem(igfsName);
+
+        try (IgfsOutputStream out = igfs.create(new IgfsPath(filePath), true)) {
+            PrintWriter wr = new PrintWriter(new OutputStreamWriter(out));
+
+            for (int i = 0; i < 1000; i++)
+                wr.println("Hello, world: " + i);
+
+            wr.flush();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
+        /** One constant. */
+        private IntWritable one = new IntWritable(1);
+
+        /** Line constant. */
+        private Text line = new Text("line");
+
+        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            ctx.write(line, one);
+        }
+    }
+
+    /**
+     * Failing mapper.
+     */
+    private static class TestFailingMapper extends Mapper<Object, Text, Text, IntWritable> {
+        @Override protected void map(Object key, Text val, Context c) throws IOException, InterruptedException {
+            throw new IOException("Test failure");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
+        /** Line constant. */
+        private Text line = new Text("line");
+
+        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
+            super.setup(ctx);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx)
+            throws IOException, InterruptedException {
+            int s = 0;
+
+            for (IntWritable val : values)
+                s += val.get();
+
+            System.out.println(">>>> Reduced: " + s);
+
+            ctx.write(line, new IntWritable(s));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 a725ddc..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(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
-            assert msg instanceof TestMessage;
-
-            msgs.add((TestMessage)msg);
-
-            receiveLatch.countDown();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onConnectionLost(GridHadoopProcessDescriptor 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 GridHadoopMessage {
-        /** 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/6423cf02/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/6423cf02/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 836cdaa..4790e63 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,75 +50,75 @@ 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");
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemSecondaryModeSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemSecondaryModeSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemClientSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemClientSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoggerStateSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemLoggerSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoggerStateSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemLoggerSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemHandshakeSelfTest.class.getName())));
+        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());
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopFileSystemsTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopFileSystemsTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopValidationSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopValidationSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopDefaultMapReducePlannerSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopJobTrackerSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopDefaultMapReducePlannerSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopJobTrackerSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopHashMapSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopDataStreamSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopConcurrentHashMultimapSelftest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopHashMapSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopDataStreamSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopConcurrentHashMultimapSelftest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSkipListSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopSkipListSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopTaskExecutionSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopTaskExecutionSelfTest.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())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopTasksV1Test.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopTasksV2Test.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopMapReduceTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopMapReduceTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopMapReduceEmbeddedSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopMapReduceEmbeddedSelfTest.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(HadoopExternalTaskExecutionSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalCommunicationSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSortingTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopSortingTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSortingExternalTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopSortingExternalTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopGroupingTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopGroupingTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopClientProtocolSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopClientProtocolEmbeddedSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopClientProtocolSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopClientProtocolEmbeddedSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopCommandLineTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopCommandLineTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSecondaryFileSystemConfigurationTest.class.getName())));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 6055db9..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
@@ -36,25 +36,25 @@ 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");
 
         suite.addTest(new TestSuite(ldr.loadClass(IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemExternalPrimarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemExternalSecondarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemExternalDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemExternalPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemExternalSecondarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopFileSystemIpcCacheSelfTest.class.getName())));
+        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());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 31aa9e5..0f0a93f 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -254,14 +254,14 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFs fileSystem(String name) {
+    @Override public IgniteFileSystem fileSystem(String name) {
         assert g != null;
 
         return g.fileSystem(name);
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFs> fileSystems() {
+    @Override public Collection<IgniteFileSystem> fileSystems() {
         assert g != null;
 
         return g.fileSystems();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index 964de1d..f0e5eba 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -1520,8 +1520,8 @@ object visor extends VisorTag {
                 val cfgs =
                     try
                         // Cache, IGFS, streamer and DR configurations should be excluded from daemon node config.
-                        spring.loadConfigurations(url, "cacheConfiguration", "igfsConfiguration", "streamerConfiguration",
-                            "drSenderHubConfiguration", "drReceiverHubConfiguration").get1()
+                        spring.loadConfigurations(url, "cacheConfiguration", "fileSystemConfiguration",
+                            "streamerConfiguration", "drSenderHubConfiguration", "drReceiverHubConfiguration").get1()
                     finally {
                         if (log4jTup != null)
                             U.removeLog4jNoOpLogger(log4jTup)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b561aab..9c6ef75 100644
--- a/pom.xml
+++ b/pom.xml
@@ -731,12 +731,12 @@
                                             <packages>org.apache.ignite.services</packages>
                                         </group>
                                         <group>
-                                            <title>Ignite File System</title>
-                                            <packages>org.apache.ignite.igfs</packages>
+                                            <title>File System APIs</title>
+                                            <packages>org.apache.ignite.igfs:org.apache.ignite.igfs.mapreduce:org.apache.ignite.igfs.mapreduce.records:org.apache.ignite.igfs.secondary</packages>
                                         </group>
                                         <group>
                                             <title>Hadoop Accelerator APIs</title>
-                                            <packages>org.apache.ignite.igfs.hadoop:org.apache.ignite.igfs.hadoop.v1:org.apache.ignite.igfs.hadoop.v2:org.apache.ignite.igfs.mapreduce:org.apache.ignite.igfs.mapreduce.records:org.apache.ignite.hadoop</packages>
+                                            <packages>org.apache.ignite.hadoop:org.apache.ignite.hadoop.fs:org.apache.ignite.hadoop.fs.v1:org.apache.ignite.hadoop.fs.v2:org.apache.ignite.hadoop.mapreduce</packages>
                                         </group>
                                         <group>
                                             <title>Streaming APIs</title>
@@ -926,12 +926,12 @@
                                             <packages>org.apache.ignite.services</packages>
                                         </group>
                                         <group>
-                                            <title>Ignite File System</title>
-                                            <packages>org.apache.ignite.igfs</packages>
+                                            <title>File System APIs</title>
+                                            <packages>org.apache.ignite.igfs:org.apache.ignite.igfs.mapreduce:org.apache.ignite.igfs.mapreduce.records:org.apache.ignite.igfs.secondary</packages>
                                         </group>
                                         <group>
                                             <title>Hadoop Accelerator APIs</title>
-                                            <packages>org.apache.ignite.igfs.hadoop:org.apache.ignite.igfs.hadoop.v1:org.apache.ignite.igfs.hadoop.v2:org.apache.ignite.igfs.mapreduce:org.apache.ignite.igfs.mapreduce.records:org.apache.ignite.hadoop</packages>
+                                            <packages>org.apache.ignite.hadoop:org.apache.ignite.hadoop.fs:org.apache.ignite.hadoop.fs.v1:org.apache.ignite.hadoop.fs.v2:org.apache.ignite.hadoop.mapreduce</packages>
                                         </group>
                                         <group>
                                             <title>Streaming APIs</title>


[37/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
new file mode 100644
index 0000000..9a6b4d8
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
@@ -0,0 +1,435 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.mapreduce;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.igfs.*;
+import org.apache.ignite.internal.processors.hadoop.planner.*;
+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.IgniteFileSystem.*;
+
+/**
+ * Default map-reduce planner implementation.
+ */
+public class IgniteHadoopMapReducePlanner implements HadoopMapReducePlanner {
+    /** Injected grid. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Logger. */
+    @SuppressWarnings("UnusedDeclaration")
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
+        @Nullable HadoopMapReducePlan 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<HadoopInputSplit>> 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<HadoopInputSplit>> mappers(Collection<ClusterNode> top, Collection<UUID> topIds,
+        Iterable<HadoopInputSplit> splits) throws IgniteCheckedException {
+        Map<UUID, Collection<HadoopInputSplit>> 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 (HadoopInputSplit split : splits) {
+            UUID nodeId = nodeForSplit(split, topIds, nodes, nodeLoads);
+
+            if (log.isDebugEnabled())
+                log.debug("Mapped split to node [split=" + split + ", nodeId=" + nodeId + ']');
+
+            Collection<HadoopInputSplit> 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(HadoopInputSplit split, Collection<UUID> topIds, Map<String, Collection<UUID>> nodes,
+        Map<UUID, Integer> nodeLoads) throws IgniteCheckedException {
+        if (split instanceof HadoopFileBlock) {
+            HadoopFileBlock split0 = (HadoopFileBlock)split;
+
+            if (IGFS_SCHEME.equalsIgnoreCase(split0.file().getScheme())) {
+                HadoopIgfsEndpoint endpoint = new HadoopIgfsEndpoint(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<HadoopInputSplit>> mappers, int reducerCnt) {
+        // Determine initial node weights.
+        int totalWeight = 0;
+
+        List<WeightedNode> nodes = new ArrayList<>(top.size());
+
+        for (ClusterNode node : top) {
+            Collection<HadoopInputSplit> 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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/package.html b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/package.html
new file mode 100644
index 0000000..e289841
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/package.html
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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. -->
+    Ignite Hadoop Accelerator map-reduce classes.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/package.html b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/package.html
new file mode 100644
index 0000000..d687e32
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/package.html
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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. -->
+    Ignite Hadoop Accelerator API.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopFileSystemWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopFileSystemWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopFileSystemWrapper.java
deleted file mode 100644
index bdab61d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopFileSystemWrapper.java
+++ /dev/null
@@ -1,414 +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.hadoop;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.ipc.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.jetbrains.annotations.*;
-import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-
-/**
- * Adapter to use any Hadoop file system {@link org.apache.hadoop.fs.FileSystem} as {@link org.apache.ignite.igfs.Igfs}.
- */
-public class IgfsHadoopFileSystemWrapper implements Igfs, AutoCloseable {
-
-    /** Hadoop file system. */
-    private final FileSystem fileSys;
-
-    /** Properties of file system */
-    private final Map<String, String> props = new HashMap<>();
-
-    /**
-     * Simple constructor that is to be used by default.
-     *
-     * @param uri URI of file system.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public IgfsHadoopFileSystemWrapper(String uri) throws IgniteCheckedException {
-        this(uri, null);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param uri URI of file system.
-     * @param cfgPath Additional path to Hadoop configuration.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public IgfsHadoopFileSystemWrapper(@Nullable String uri, @Nullable String cfgPath) throws IgniteCheckedException {
-        try {
-            SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(uri, cfgPath);
-
-            fileSys = secProvider.createFileSystem();
-
-            uri = secProvider.uri().toString();
-
-            if (!uri.endsWith("/"))
-                uri += "/";
-
-            props.put(SECONDARY_FS_CONFIG_PATH, cfgPath);
-            props.put(SECONDARY_FS_URI, uri);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /**
-     * Convert IGFS path into Hadoop path.
-     *
-     * @param path IGFS path.
-     * @return Hadoop path.
-     */
-    private Path convert(IgfsPath path) {
-        URI uri = fileSys.getUri();
-
-        return new Path(uri.getScheme(), uri.getAuthority(), path.toString());
-    }
-
-    /**
-     * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception.
-     *
-     * @param e Exception to check.
-     * @param detailMsg Detailed error message.
-     * @return Appropriate exception.
-     */
-    private IgfsException handleSecondaryFsError(IOException e, String detailMsg) {
-        boolean wrongVer = X.hasCause(e, RemoteException.class) ||
-            (e.getMessage() != null && e.getMessage().contains("Failed on local"));
-
-        IgfsException igfsErr = !wrongVer ? cast(detailMsg, e) :
-            new IgfsInvalidHdfsVersionException("HDFS version you are connecting to differs from local " +
-                "version.", e);
-
-        return igfsErr;
-    }
-
-    /**
-     * Cast IO exception to IGFS exception.
-     *
-     * @param e IO exception.
-     * @return IGFS exception.
-     */
-    public static IgfsException cast(String msg, IOException e) {
-        if (e instanceof FileNotFoundException)
-            return new IgfsFileNotFoundException(e);
-        else if (e instanceof ParentNotDirectoryException)
-            return new IgfsParentNotDirectoryException(msg, e);
-        else if (e instanceof PathIsNotEmptyDirectoryException)
-            return new IgfsDirectoryNotEmptyException(e);
-        else if (e instanceof PathExistsException)
-            return new IgfsPathAlreadyExistsException(msg, e);
-        else
-            return new IgfsException(msg, e);
-    }
-
-    /**
-     * Convert Hadoop FileStatus properties to map.
-     *
-     * @param status File status.
-     * @return IGFS attributes.
-     */
-    private static Map<String, String> properties(FileStatus status) {
-        FsPermission perm = status.getPermission();
-
-        if (perm == null)
-            perm = FsPermission.getDefault();
-
-        return F.asMap(PROP_PERMISSION, String.format("%04o", perm.toShort()), PROP_USER_NAME, status.getOwner(),
-            PROP_GROUP_NAME, status.getGroup());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean exists(IgfsPath path) {
-        try {
-            return fileSys.exists(convert(path));
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to check file existence [path=" + path + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
-        IgfsHadoopFSProperties props0 = new IgfsHadoopFSProperties(props);
-
-        try {
-            if (props0.userName() != null || props0.groupName() != null)
-                fileSys.setOwner(convert(path), props0.userName(), props0.groupName());
-
-            if (props0.permission() != null)
-                fileSys.setPermission(convert(path), props0.permission());
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to update file properties [path=" + path + "]");
-        }
-
-        //Result is not used in case of secondary FS.
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rename(IgfsPath src, IgfsPath dest) {
-        // Delegate to the secondary file system.
-        try {
-            if (!fileSys.rename(convert(src), convert(dest)))
-                throw new IgfsException("Failed to rename (secondary file system returned false) " +
-                    "[src=" + src + ", dest=" + dest + ']');
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to rename file [src=" + src + ", dest=" + dest + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(IgfsPath path, boolean recursive) {
-        try {
-            return fileSys.delete(convert(path), recursive);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to delete file [path=" + path + ", recursive=" + recursive + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(IgfsPath path) {
-        try {
-            if (!fileSys.mkdirs(convert(path)))
-                throw new IgniteException("Failed to make directories [path=" + path + "]");
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
-        try {
-            if (!fileSys.mkdirs(convert(path), new IgfsHadoopFSProperties(props).permission()))
-                throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]");
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + ", props=" + props + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
-        try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
-
-            if (statuses == null)
-                throw new IgfsFileNotFoundException("Failed to list files (path not found): " + path);
-
-            Collection<IgfsPath> res = new ArrayList<>(statuses.length);
-
-            for (FileStatus status : statuses)
-                res.add(new IgfsPath(path, status.getPath().getName()));
-
-            return res;
-        }
-        catch (FileNotFoundException ignored) {
-            throw new IgfsFileNotFoundException("Failed to list files (path not found): " + path);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
-        try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
-
-            if (statuses == null)
-                throw new IgfsFileNotFoundException("Failed to list files (path not found): " + path);
-
-            Collection<IgfsFile> res = new ArrayList<>(statuses.length);
-
-            for (FileStatus status : statuses) {
-                IgfsFileInfo fsInfo = status.isDirectory() ? new IgfsFileInfo(true, properties(status)) :
-                    new IgfsFileInfo((int)status.getBlockSize(), status.getLen(), null, null, false,
-                    properties(status));
-
-                res.add(new IgfsFileImpl(new IgfsPath(path, status.getPath().getName()), fsInfo, 1));
-            }
-
-            return res;
-        }
-        catch (FileNotFoundException ignored) {
-            throw new IgfsFileNotFoundException("Failed to list files (path not found): " + path);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsReader open(IgfsPath path, int bufSize) {
-        return new IgfsHadoopReader(fileSys, convert(path), bufSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream create(IgfsPath path, boolean overwrite) {
-        try {
-            return fileSys.create(convert(path), overwrite);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + "]");
-        }
-    }
-
-    /** {@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());
-
-        try {
-            return fileSys.create(convert(path), props0.permission(), overwrite, bufSize, (short)replication, blockSize,
-                null);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", props=" + props +
-                ", overwrite=" + overwrite + ", bufSize=" + bufSize + ", replication=" + replication +
-                ", blockSize=" + blockSize + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) {
-        try {
-            return fileSys.append(convert(path), bufSize);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to append file [path=" + path + ", bufSize=" + bufSize + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(final IgfsPath path) {
-        try {
-            final FileStatus status = fileSys.getFileStatus(convert(path));
-
-            if (status == null)
-                return null;
-
-            final Map<String, String> props = properties(status);
-
-            return new IgfsFile() {
-                @Override public IgfsPath path() {
-                    return path;
-                }
-
-                @Override public boolean isFile() {
-                    return status.isFile();
-                }
-
-                @Override public boolean isDirectory() {
-                    return status.isDirectory();
-                }
-
-                @Override public int blockSize() {
-                    return (int)status.getBlockSize();
-                }
-
-                @Override public long groupBlockSize() {
-                    return status.getBlockSize();
-                }
-
-                @Override public long accessTime() {
-                    return status.getAccessTime();
-                }
-
-                @Override public long modificationTime() {
-                    return status.getModificationTime();
-                }
-
-                @Override public String property(String name) throws IllegalArgumentException {
-                    String val = props.get(name);
-
-                    if (val ==  null)
-                        throw new IllegalArgumentException("File property not found [path=" + path + ", name=" + name + ']');
-
-                    return val;
-                }
-
-                @Nullable @Override public String property(String name, @Nullable String dfltVal) {
-                    String val = props.get(name);
-
-                    return val == null ? dfltVal : val;
-                }
-
-                @Override public long length() {
-                    return status.getLen();
-                }
-
-                /** {@inheritDoc} */
-                @Override public Map<String, String> properties() {
-                    return props;
-                }
-            };
-
-        }
-        catch (FileNotFoundException ignore) {
-            return null;
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to get file status [path=" + path + "]");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public long usedSpaceSize() {
-        try {
-            return fileSys.getContentSummary(new Path(fileSys.getUri())).getSpaceConsumed();
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to get used space size of file system.");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Map<String, String> properties() {
-        return props;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
-        try {
-            fileSys.close();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopParameters.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopParameters.java b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopParameters.java
deleted file mode 100644
index 6237dd4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/IgfsHadoopParameters.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.igfs.hadoop;
-
-/**
- * This class lists parameters that can be specified in Hadoop configuration.
- * Hadoop configuration can be specified in {@code core-site.xml} file
- * or passed to map-reduce task directly when using Hadoop driver for IGFS file system:
- * <ul>
- *     <li>
- *         {@code fs.igfs.[name].open.sequential_reads_before_prefetch} - this parameter overrides
- *         the one specified in {@link org.apache.ignite.configuration.IgfsConfiguration#getSequentialReadsBeforePrefetch()}
- *         IGFS data node configuration property.
- *     </li>
- *     <li>
- *         {@code fs.igfs.[name].log.enabled} - specifies whether IGFS sampling logger is enabled. If
- *         {@code true}, then all file system operations will be logged to a file.
- *     </li>
- *     <li>{@code fs.igfs.[name].log.dir} - specifies log directory where sampling log files should be placed.</li>
- *     <li>
- *         {@code fs.igfs.[name].log.batch_size} - specifies how many log entries are accumulated in a batch before
- *         it gets flushed to log file. Higher values will imply greater performance, but will increase delay
- *         before record appears in the log file.
- *     </li>
- *     <li>
- *         {@code fs.igfs.[name].colocated.writes} - specifies whether written files should be colocated on data
- *         node to which client is connected. If {@code true}, file will not be distributed and will be written
- *         to a single data node. Default value is {@code true}.
- *     </li>
- *     <li>
- *         {@code fs.igfs.prefer.local.writes} - specifies whether file preferably should be written to
- *         local data node if it has enough free space. After some time it can be redistributed across nodes though.
- *     </li>
- * </ul>
- * Where {@code [name]} is file system endpoint which you specify in file system URI authority part. E.g. in
- * case your file system URI is {@code igfs://127.0.0.1:10500} then {@code name} will be {@code 127.0.0.1:10500}.
- * <p>
- * Sample configuration that can be placed to {@code core-site.xml} file:
- * <pre name="code" class="xml">
- *     &lt;property&gt;
- *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.enabled&lt;/name&gt;
- *         &lt;value&gt;true&lt;/value&gt;
- *     &lt;/property&gt;
- *     &lt;property&gt;
- *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.dir&lt;/name&gt;
- *         &lt;value&gt;/home/apache/ignite/log/sampling&lt;/value&gt;
- *     &lt;/property&gt;
- *     &lt;property&gt;
- *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.batch_size&lt;/name&gt;
- *         &lt;value&gt;16&lt;/value&gt;
- *     &lt;/property&gt;
- * </pre>
- * Parameters could also be specified per mapreduce job, e.g.
- * <pre name="code" class="bash">
- * hadoop jar myjarfile.jar MyMapReduceJob -Dfs.igfs.open.sequential_reads_before_prefetch=4
- * </pre>
- * If you want to use these parameters in code, then you have to substitute you file system name in it. The easiest
- * way to do that is {@code String.format(PARAM_IGFS_COLOCATED_WRITES, [name])}.
- */
-public class IgfsHadoopParameters {
-    /** Parameter name for control over file colocation write mode. */
-    public static final String PARAM_IGFS_COLOCATED_WRITES = "fs.igfs.%s.colocated.writes";
-
-    /** Parameter name for custom sequential reads before prefetch value. */
-    public static final String PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH =
-        "fs.igfs.%s.open.sequential_reads_before_prefetch";
-
-    /** Parameter name for client logger directory. */
-    public static final String PARAM_IGFS_LOG_DIR = "fs.igfs.%s.log.dir";
-
-    /** Parameter name for log batch size. */
-    public static final String PARAM_IGFS_LOG_BATCH_SIZE = "fs.igfs.%s.log.batch_size";
-
-    /** Parameter name for log enabled flag. */
-    public static final String PARAM_IGFS_LOG_ENABLED = "fs.igfs.%s.log.enabled";
-
-    /** Parameter name for prefer local writes flag. */
-    public static final String PARAM_IGFS_PREFER_LOCAL_WRITES = "fs.igfs.prefer.local.writes";
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/package.html b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/package.html
deleted file mode 100644
index 137055b..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/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 common files for Hadoop 1.x and Hadoop 2.x distros.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/IgfsHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/IgfsHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/IgfsHadoopFileSystem.java
deleted file mode 100644
index c4d2f5e..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/IgfsHadoopFileSystem.java
+++ /dev/null
@@ -1,1234 +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.hadoop.v1;
-
-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.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-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.net.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-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.igfs.hadoop.IgfsHadoopParameters.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
-import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
-
-/**
- * {@code IGFS} Hadoop 1.x file system driver over file system API. To use
- * {@code IGFS} as Hadoop file system, you should configure this class
- * in Hadoop's {@code core-site.xml} as follows:
- * <pre name="code" class="xml">
- *  &lt;property&gt;
- *      &lt;name&gt;fs.default.name&lt;/name&gt;
- *      &lt;value&gt;igfs://ipc&lt;/value&gt;
- *  &lt;/property&gt;
- *
- *  &lt;property&gt;
- *      &lt;name&gt;fs.igfs.impl&lt;/name&gt;
- *      &lt;value&gt;org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem&lt;/value&gt;
- *  &lt;/property&gt;
- * </pre>
- * You should also add Ignite JAR and all libraries to Hadoop classpath. To
- * do this, add following lines to {@code conf/hadoop-env.sh} script in Hadoop
- * distribution:
- * <pre name="code" class="bash">
- * export IGNITE_HOME=/path/to/Ignite/distribution
- * export HADOOP_CLASSPATH=$IGNITE_HOME/ignite*.jar
- *
- * for f in $IGNITE_HOME/libs/*.jar; do
- *  export HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$f;
- * done
- * </pre>
- * <h1 class="header">Data vs Clients Nodes</h1>
- * Hadoop needs to use its FileSystem remotely from client nodes as well as directly on
- * data nodes. Client nodes are responsible for basic file system operations as well as
- * accessing data nodes remotely. Usually, client nodes are started together
- * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
- * started together with Hadoop {@code task-tracker} processes.
- * <p>
- * For sample client and data node configuration refer to {@code config/hadoop/default-config-client.xml}
- * and {@code config/hadoop/default-config.xml} configuration files in Ignite installation.
- */
-public class IgfsHadoopFileSystem extends FileSystem {
-    /** Internal property to indicate management connection. */
-    public static final String IGFS_MANAGEMENT = "fs.igfs.management.connection";
-
-    /** Empty array of file block locations. */
-    private static final BlockLocation[] EMPTY_BLOCK_LOCATIONS = new BlockLocation[0];
-
-    /** Empty array of file statuses. */
-    public static final FileStatus[] EMPTY_FILE_STATUS = new FileStatus[0];
-
-    /** Ensures that close routine is invoked at most once. */
-    private final AtomicBoolean closeGuard = new AtomicBoolean();
-
-    /** Grid remote client. */
-    private IgfsHadoopWrapper rmtClient;
-
-    /** 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();
-        }
-    };
-
-    /** Default replication factor. */
-    private short dfltReplication;
-
-    /** Base file system uri. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private URI uri;
-
-    /** Authority. */
-    private String uriAuthority;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Secondary URI string. */
-    private URI secondaryUri;
-
-    /** IGFS mode resolver. */
-    private IgfsModeResolver modeRslvr;
-
-    /** Secondary file system instance. */
-    private FileSystem secondaryFs;
-
-    /** Management connection flag. */
-    private boolean mgmt;
-
-    /** Whether custom sequential reads before prefetch value is provided. */
-    private boolean seqReadsBeforePrefetchOverride;
-
-    /** IGFS group block size. */
-    private long igfsGrpBlockSize;
-
-    /** Flag that controls whether file writes should be colocated. */
-    private boolean colocateFileWrites;
-
-    /** Prefer local writes. */
-    private boolean preferLocFileWrites;
-
-    /** Custom-provided sequential reads before prefetch. */
-    private int seqReadsBeforePrefetch;
-
-    /** The cache was disabled when the instance was creating. */
-    private boolean cacheEnabled;
-
-    /** {@inheritDoc} */
-    @Override public URI getUri() {
-        if (uri == null)
-            throw new IllegalStateException("URI is null (was IgfsHadoopFileSystem properly initialized?).");
-
-        return uri;
-    }
-
-    /**
-     * Enter busy state.
-     *
-     * @throws IOException If file system is stopped.
-     */
-    private void enterBusy() throws IOException {
-        if (closeGuard.get())
-            throw new IOException("File system is stopped.");
-    }
-
-    /**
-     * Leave busy state.
-     */
-    private void leaveBusy() {
-        // No-op.
-    }
-
-    /**
-     * Public setter that can be used by direct users of FS or Visor.
-     *
-     * @param colocateFileWrites Whether all ongoing file writes should be colocated.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public void colocateFileWrites(boolean colocateFileWrites) {
-        this.colocateFileWrites = colocateFileWrites;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initialize(URI name, Configuration cfg) throws IOException {
-        enterBusy();
-
-        try {
-            if (rmtClient != null)
-                throw new IOException("File system is already initialized: " + rmtClient);
-
-            A.notNull(name, "name");
-            A.notNull(cfg, "cfg");
-
-            super.initialize(name, cfg);
-
-            setConf(cfg);
-
-            String disableCacheName = String.format("fs.%s.impl.disable.cache", name.getScheme());
-
-            cacheEnabled = !cfg.getBoolean(disableCacheName, false);
-
-            mgmt = cfg.getBoolean(IGFS_MANAGEMENT, false);
-
-            if (!IGFS_SCHEME.equals(name.getScheme()))
-                throw new IOException("Illegal file system URI [expected=" + IGFS_SCHEME +
-                    "://[name]/[optional_path], actual=" + name + ']');
-
-            uri = name;
-
-            uriAuthority = uri.getAuthority();
-
-            setUser(cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
-
-            // Override sequential reads before prefetch if needed.
-            seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
-
-            if (seqReadsBeforePrefetch > 0)
-                seqReadsBeforePrefetchOverride = true;
-
-            // In Ignite replication factor is controlled by data cache affinity.
-            // We use replication factor to force the whole file to be stored on local node.
-            dfltReplication = (short)cfg.getInt("dfs.replication", 3);
-
-            // Get file colocation control flag.
-            colocateFileWrites = parameter(cfg, PARAM_IGFS_COLOCATED_WRITES, uriAuthority, false);
-            preferLocFileWrites = cfg.getBoolean(PARAM_IGFS_PREFER_LOCAL_WRITES, false);
-
-            // Get log directory.
-            String logDirCfg = parameter(cfg, PARAM_IGFS_LOG_DIR, uriAuthority, DFLT_IGFS_LOG_DIR);
-
-            File logDirFile = U.resolveIgnitePath(logDirCfg);
-
-            String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
-
-            rmtClient = new IgfsHadoopWrapper(uriAuthority, logDir, cfg, LOG);
-
-            // Handshake.
-            IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
-
-            igfsGrpBlockSize = handshake.blockSize();
-
-            IgfsPaths paths = handshake.secondaryPaths();
-
-            // Initialize client logger.
-            Boolean logEnabled = parameter(cfg, PARAM_IGFS_LOG_ENABLED, uriAuthority, false);
-
-            if (handshake.sampling() != null ? handshake.sampling() : logEnabled) {
-                // Initiate client logger.
-                if (logDir == null)
-                    throw new IOException("Failed to resolve log directory: " + logDirCfg);
-
-                Integer batchSize = parameter(cfg, PARAM_IGFS_LOG_BATCH_SIZE, uriAuthority, DFLT_IGFS_LOG_BATCH_SIZE);
-
-                clientLog = IgfsLogger.logger(uriAuthority, handshake.igfsName(), logDir, batchSize);
-            }
-            else
-                clientLog = IgfsLogger.disabledLogger();
-
-            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
-
-            boolean initSecondary = paths.defaultMode() == PROXY;
-
-            if (!initSecondary && paths.pathModes() != null && !paths.pathModes().isEmpty()) {
-                for (T2<IgfsPath, IgfsMode> pathMode : paths.pathModes()) {
-                    IgfsMode mode = pathMode.getValue();
-
-                    if (mode == PROXY) {
-                        initSecondary = true;
-
-                        break;
-                    }
-                }
-            }
-
-            if (initSecondary) {
-                Map<String, String> props = paths.properties();
-
-                String secUri = props.get(SECONDARY_FS_URI);
-                String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-
-                try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
-
-                    secondaryFs = secProvider.createFileSystem();
-                    secondaryUri = secProvider.uri();
-                }
-                catch (IOException e) {
-                    if (!mgmt)
-                        throw new IOException("Failed to connect to the secondary file system: " + secUri, e);
-                    else
-                        LOG.warn("Visor failed to create secondary file system (operations on paths with PROXY mode " +
-                            "will have no effect): " + e.getMessage());
-                }
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPath(Path path) {
-        URI uri = path.toUri();
-
-        if (uri.isAbsolute()) {
-            if (!F.eq(uri.getScheme(), IGFS_SCHEME))
-                throw new InvalidPathException("Wrong path scheme [expected=" + IGFS_SCHEME + ", actual=" +
-                    uri.getAuthority() + ']');
-
-            if (!F.eq(uri.getAuthority(), uriAuthority))
-                throw new InvalidPathException("Wrong path authority [expected=" + uriAuthority + ", actual=" +
-                    uri.getAuthority() + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public short getDefaultReplication() {
-        return dfltReplication;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void finalize() throws Throwable {
-        super.finalize();
-
-        close0();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        if (cacheEnabled && get(getUri(), getConf()) == this)
-            return;
-
-        close0();
-    }
-
-    /**
-     * Closes file system.
-     *
-     * @throws IOException If failed.
-     */
-    private void close0() throws IOException {
-        if (closeGuard.compareAndSet(false, true)) {
-            if (LOG.isDebugEnabled())
-                LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
-
-            if (rmtClient == null)
-                return;
-
-            super.close();
-
-            rmtClient.close(false);
-
-            if (clientLog.isLogEnabled())
-                clientLog.close();
-
-            if (secondaryFs != null)
-                U.closeQuiet(secondaryFs);
-
-            // Reset initialized resources.
-            uri = null;
-            rmtClient = null;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setTimes(Path p, long mtime, long atime) throws IOException {
-        enterBusy();
-
-        try {
-            A.notNull(p, "p");
-
-            if (mode(p) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setTimes(toSecondary(p), mtime, atime);
-            }
-            else {
-                IgfsPath path = convert(p);
-
-                rmtClient.setTimes(path, atime, mtime);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setPermission(Path p, FsPermission perm) throws IOException {
-        enterBusy();
-
-        try {
-            A.notNull(p, "p");
-
-            if (mode(p) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setPermission(toSecondary(p), perm);
-            }
-            else if (rmtClient.update(convert(p), permission(perm)) == null) {
-                throw new IOException("Failed to set file permission (file not found?)" +
-                    " [path=" + p + ", perm=" + perm + ']');
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setOwner(Path p, String username, String grpName) throws IOException {
-        A.notNull(p, "p");
-        A.notNull(username, "username");
-        A.notNull(grpName, "grpName");
-
-        enterBusy();
-
-        try {
-            if (mode(p) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    // No-op for management connection.
-                    return;
-                }
-
-                secondaryFs.setOwner(toSecondary(p), username, grpName);
-            }
-            else if (rmtClient.update(convert(p), F.asMap(PROP_USER_NAME, username, PROP_GROUP_NAME, grpName)) == null)
-                throw new IOException("Failed to set file permission (file not found?)" +
-                    " [path=" + p + ", userName=" + username + ", groupName=" + grpName + ']');
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FSDataInputStream open(Path f, int bufSize) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to open file (secondary file system is not initialized): " + f);
-                }
-
-                FSDataInputStream is = secondaryFs.open(toSecondary(f), bufSize);
-
-                if (clientLog.isLogEnabled()) {
-                    // At this point we do not know file size, so we perform additional request to remote FS to get it.
-                    FileStatus status = secondaryFs.getFileStatus(toSecondary(f));
-
-                    long size = status != null ? status.getLen() : -1;
-
-                    long logId = IgfsLogger.nextId();
-
-                    clientLog.logOpen(logId, path, PROXY, bufSize, size);
-
-                    return new FSDataInputStream(new IgfsHadoopProxyInputStream(is, clientLog, logId));
-                }
-                else
-                    return is;
-            }
-            else {
-                IgfsHadoopStreamDelegate stream = seqReadsBeforePrefetchOverride ?
-                    rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
-
-                    clientLog.logOpen(logId, path, mode, bufSize, stream.length());
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
-                        ", bufSize=" + bufSize + ']');
-
-                IgfsHadoopInputStream igfsIn = new IgfsHadoopInputStream(stream, stream.length(),
-                    bufSize, LOG, clientLog, logId);
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
-
-                return new FSDataInputStream(igfsIn);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public FSDataOutputStream create(Path f, FsPermission perm, boolean overwrite, int bufSize,
-        short replication, long blockSize, Progressable progress) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        OutputStream out = null;
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (LOG.isDebugEnabled())
-                LOG.debug("Opening output stream in create [thread=" + Thread.currentThread().getName() + "path=" +
-                    path + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ']');
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to create file (secondary file system is not initialized): " + f);
-                }
-
-                FSDataOutputStream os =
-                    secondaryFs.create(toSecondary(f), perm, overwrite, bufSize, replication, blockSize, progress);
-
-                if (clientLog.isLogEnabled()) {
-                    long logId = IgfsLogger.nextId();
-
-                    clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
-
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(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,
-                    replication, blockSize, F.asMap(PROP_PERMISSION, toString(perm),
-                    PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites)));
-
-                assert stream != null;
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
-
-                    clientLog.logCreate(logId, path, mode, overwrite, bufSize, replication, blockSize);
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
-
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG, clientLog,
-                    logId);
-
-                bufSize = Math.max(64 * 1024, bufSize);
-
-                out = new BufferedOutputStream(igfsOut, bufSize);
-
-                FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
-
-                // Mark stream created successfully.
-                out = null;
-
-                return res;
-            }
-        }
-        finally {
-            // Close if failed during stream creation.
-            if (out != null)
-                U.closeQuiet(out);
-
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public FSDataOutputStream append(Path f, int bufSize, Progressable progress) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (LOG.isDebugEnabled())
-                LOG.debug("Opening output stream in append [thread=" + Thread.currentThread().getName() +
-                    ", path=" + path + ", bufSize=" + bufSize + ']');
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to append file (secondary file system is not initialized): " + f);
-                }
-
-                FSDataOutputStream os = secondaryFs.append(toSecondary(f), bufSize, progress);
-
-                if (clientLog.isLogEnabled()) {
-                    long logId = IgfsLogger.nextId();
-
-                    clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
-
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(os, clientLog, logId));
-                }
-                else
-                    return os;
-            }
-            else {
-                IgfsHadoopStreamDelegate stream = rmtClient.append(path, false, null);
-
-                assert stream != null;
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
-
-                    clientLog.logAppend(logId, path, mode, bufSize);
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
-
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG, clientLog,
-                    logId);
-
-                bufSize = Math.max(64 * 1024, bufSize);
-
-                BufferedOutputStream out = new BufferedOutputStream(igfsOut, bufSize);
-
-                return new FSDataOutputStream(out, null, 0);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean rename(Path src, Path dst) throws IOException {
-        A.notNull(src, "src");
-        A.notNull(dst, "dst");
-
-        enterBusy();
-
-        try {
-            IgfsPath srcPath = convert(src);
-            IgfsPath dstPath = convert(dst);
-            IgfsMode mode = mode(srcPath);
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logRename(srcPath, PROXY, dstPath);
-
-                return secondaryFs.rename(toSecondary(src), toSecondary(dst));
-            }
-            else {
-                // Will throw exception if failed.
-                rmtClient.rename(srcPath, dstPath);
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logRename(srcPath, mode, dstPath);
-
-                return true;
-            }
-        }
-        catch (IOException e) {
-            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
-            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
-                !X.hasCause(e.getCause(), IgfsException.class)))
-                throw e;
-            else
-                return false;
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public boolean delete(Path f) throws IOException {
-        return delete(f, false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(Path f, boolean recursive) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logDelete(path, PROXY, recursive);
-
-                return secondaryFs.delete(toSecondary(f), recursive);
-            }
-            else {
-                // Will throw exception if delete failed.
-                boolean res = rmtClient.delete(path, recursive);
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logDelete(path, mode, recursive);
-
-                return res;
-            }
-        }
-        catch (IOException e) {
-            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
-            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
-                !X.hasCause(e.getCause(), IgfsException.class)))
-                throw e;
-            else
-                return false;
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus[] listStatus(Path f) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return EMPTY_FILE_STATUS;
-                }
-
-                FileStatus[] arr = secondaryFs.listStatus(toSecondary(f));
-
-                if (arr == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = toPrimary(arr[i]);
-
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
-
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, PROXY, fileArr);
-                }
-
-                return arr;
-            }
-            else {
-                Collection<IgfsFile> list = rmtClient.listFiles(path);
-
-                if (list == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
-
-                List<IgfsFile> files = new ArrayList<>(list);
-
-                FileStatus[] arr = new FileStatus[files.size()];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(files.get(i));
-
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
-
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, mode, fileArr);
-                }
-
-                return arr;
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
-
-        return path.makeQualified(getUri(), null);
-    }
-
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setWorkingDirectory(Path newPath) {
-        if (newPath == null) {
-            Path homeDir = getHomeDirectory();
-
-            if (secondaryFs != null)
-                secondaryFs.setWorkingDirectory(toSecondary(homeDir));
-
-            workingDir.set(homeDir);
-        }
-        else {
-            Path fixedNewPath = fixRelativePart(newPath);
-
-            String res = fixedNewPath.toUri().getPath();
-
-            if (!DFSUtil.isValidName(res))
-                throw new IllegalArgumentException("Invalid DFS directory name " + res);
-
-            if (secondaryFs != null)
-                secondaryFs.setWorkingDirectory(toSecondary(fixedNewPath));
-
-            workingDir.set(fixedNewPath);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getWorkingDirectory() {
-        return workingDir.get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean mkdirs(Path f, FsPermission perm) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = mode(path);
-
-            if (mode == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return false;
-                }
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, PROXY);
-
-                return secondaryFs.mkdirs(toSecondary(f), perm);
-            }
-            else {
-                boolean mkdirRes = rmtClient.mkdirs(path, permission(perm));
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, mode);
-
-                return mkdirRes;
-            }
-        }
-        catch (IOException e) {
-            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
-            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
-                !X.hasCause(e.getCause(), IgfsException.class)))
-                throw e;
-            else
-                return false;
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus getFileStatus(Path f) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            if (mode(f) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to get file status (secondary file system is not initialized): " + f);
-                }
-
-                return toPrimary(secondaryFs.getFileStatus(toSecondary(f)));
-            }
-            else {
-                IgfsFile info = rmtClient.info(convert(f));
-
-                if (info == null)
-                    throw new FileNotFoundException("File not found: " + f);
-
-                return convert(info);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ContentSummary getContentSummary(Path f) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            if (mode(f) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    throw new IOException("Failed to get content summary (secondary file system is not initialized): " +
-                        f);
-                }
-
-                return secondaryFs.getContentSummary(toSecondary(f));
-            }
-            else {
-                IgfsPathSummary sum = rmtClient.contentSummary(convert(f));
-
-                return new ContentSummary(sum.totalLength(), sum.filesCount(), sum.directoriesCount(),
-                    -1, sum.totalLength(), rmtClient.fsStatus().spaceTotal());
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public BlockLocation[] getFileBlockLocations(FileStatus status, long start, long len) throws IOException {
-        A.notNull(status, "status");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(status.getPath());
-
-            if (mode(status.getPath()) == PROXY) {
-                if (secondaryFs == null) {
-                    assert mgmt;
-
-                    return EMPTY_BLOCK_LOCATIONS;
-                }
-
-                Path secPath = toSecondary(status.getPath());
-
-                return secondaryFs.getFileBlockLocations(secondaryFs.getFileStatus(secPath), start, len);
-            }
-            else {
-                long now = System.currentTimeMillis();
-
-                List<IgfsBlockLocation> affinity = new ArrayList<>(rmtClient.affinity(path, start, len));
-
-                BlockLocation[] arr = new BlockLocation[affinity.size()];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(affinity.get(i));
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
-                        (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
-
-                return arr;
-            }
-        }
-        catch (FileNotFoundException ignored) {
-            return EMPTY_BLOCK_LOCATIONS;
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public long getDefaultBlockSize() {
-        return igfsGrpBlockSize;
-    }
-
-    /**
-     * Resolve path mode.
-     *
-     * @param path HDFS path.
-     * @return Path mode.
-     */
-    public IgfsMode mode(Path path) {
-        return mode(convert(path));
-    }
-
-    /**
-     * Resolve path mode.
-     *
-     * @param path IGFS path.
-     * @return Path mode.
-     */
-    public IgfsMode mode(IgfsPath path) {
-        return modeRslvr.resolveMode(path);
-    }
-
-    /**
-     * Convert the given path to path acceptable by the primary file system.
-     *
-     * @param path Path.
-     * @return Primary file system path.
-     */
-    private Path toPrimary(Path path) {
-        return convertPath(path, uri);
-    }
-
-    /**
-     * Convert the given path to path acceptable by the secondary file system.
-     *
-     * @param path Path.
-     * @return Secondary file system path.
-     */
-    private Path toSecondary(Path path) {
-        assert secondaryFs != null;
-        assert secondaryUri != null;
-
-        return convertPath(path, secondaryUri);
-    }
-
-    /**
-     * Convert path using the given new URI.
-     *
-     * @param path Old path.
-     * @param newUri New URI.
-     * @return New path.
-     */
-    private Path convertPath(Path path, URI newUri) {
-        assert newUri != null;
-
-        if (path != null) {
-            URI pathUri = path.toUri();
-
-            try {
-                return new Path(new URI(pathUri.getScheme() != null ? newUri.getScheme() : null,
-                    pathUri.getAuthority() != null ? newUri.getAuthority() : null, pathUri.getPath(), null, null));
-            }
-            catch (URISyntaxException e) {
-                throw new IgniteException("Failed to construct secondary file system path from the primary file " +
-                    "system path: " + path, e);
-            }
-        }
-        else
-            return null;
-    }
-
-    /**
-     * Convert a file status obtained from the secondary file system to a status of the primary file system.
-     *
-     * @param status Secondary file system status.
-     * @return Primary file system status.
-     */
-    @SuppressWarnings("deprecation")
-    private FileStatus toPrimary(FileStatus status) {
-        return status != null ? new FileStatus(status.getLen(), status.isDir(), status.getReplication(),
-            status.getBlockSize(), status.getModificationTime(), status.getAccessTime(), status.getPermission(),
-            status.getOwner(), status.getGroup(), toPrimary(status.getPath())) : null;
-    }
-
-    /**
-     * Convert IGFS path into Hadoop path.
-     *
-     * @param path IGFS path.
-     * @return Hadoop path.
-     */
-    private Path convert(IgfsPath path) {
-        return new Path(IGFS_SCHEME, uriAuthority, path.toString());
-    }
-
-    /**
-     * Convert Hadoop path into IGFS path.
-     *
-     * @param path Hadoop path.
-     * @return IGFS path.
-     */
-    @Nullable private IgfsPath convert(@Nullable Path path) {
-        if (path == null)
-            return null;
-
-        return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
-            new IgfsPath(convert(workingDir.get()), path.toUri().getPath());
-    }
-
-    /**
-     * Convert IGFS affinity block location into Hadoop affinity block location.
-     *
-     * @param block IGFS affinity block location.
-     * @return Hadoop affinity block location.
-     */
-    private BlockLocation convert(IgfsBlockLocation block) {
-        Collection<String> names = block.names();
-        Collection<String> hosts = block.hosts();
-
-        return new BlockLocation(
-            names.toArray(new String[names.size()]) /* hostname:portNumber of data nodes */,
-            hosts.toArray(new String[hosts.size()]) /* hostnames of data nodes */,
-            block.start(), block.length()
-        ) {
-            @Override public String toString() {
-                try {
-                    return "BlockLocation [offset=" + getOffset() + ", length=" + getLength() +
-                        ", hosts=" + Arrays.asList(getHosts()) + ", names=" + Arrays.asList(getNames()) + ']';
-                }
-                catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Convert IGFS file information into Hadoop file status.
-     *
-     * @param file IGFS file information.
-     * @return Hadoop file status.
-     */
-    @SuppressWarnings("deprecation")
-    private FileStatus convert(IgfsFile file) {
-        return new FileStatus(file.length(), file.isDirectory(), getDefaultReplication(),
-            file.groupBlockSize(), file.modificationTime(), file.accessTime(), permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME), file.property(PROP_GROUP_NAME, "users"),
-            convert(file.path())) {
-            @Override public String toString() {
-                return "FileStatus [path=" + getPath() + ", isDir=" + isDir() + ", len=" + getLen() +
-                    ", mtime=" + getModificationTime() + ", atime=" + getAccessTime() + ']';
-            }
-        };
-    }
-
-    /**
-     * Convert Hadoop permission into IGFS file attribute.
-     *
-     * @param perm Hadoop permission.
-     * @return IGFS attributes.
-     */
-    private Map<String, String> permission(FsPermission perm) {
-        if (perm == null)
-            perm = FsPermission.getDefault();
-
-        return F.asMap(PROP_PERMISSION, toString(perm));
-    }
-
-    /**
-     * @param perm Permission.
-     * @return String.
-     */
-    private static String toString(FsPermission perm) {
-        return String.format("%04o", perm.toShort());
-    }
-
-    /**
-     * Convert IGFS file attributes into Hadoop permission.
-     *
-     * @param file File info.
-     * @return Hadoop permission.
-     */
-    private FsPermission permission(IgfsFile file) {
-        String perm = file.property(PROP_PERMISSION, null);
-
-        if (perm == null)
-            return FsPermission.getDefault();
-
-        try {
-            return new FsPermission((short)Integer.parseInt(perm, 8));
-        }
-        catch (NumberFormatException ignore) {
-            return FsPermission.getDefault();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsHadoopFileSystem.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/package.html b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/package.html
deleted file mode 100644
index 4b62db1..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v1/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 Hadoop 1.x <code>FileSystem</code> wrapper for Ignite file system.
-</body>
-</html>


[55/58] [abbrv] incubator-ignite git commit: # sprint-2 Fixed load schemas and tables for MySql.

Posted by yz...@apache.org.
# sprint-2 Fixed load schemas and tables for MySql.


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

Branch: refs/heads/ignite-313
Commit: 49af1cf28a0c15d208375f821c2c262794704c9b
Parents: e4d0476
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Mar 5 15:40:09 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Mar 5 15:40:09 2015 +0700

----------------------------------------------------------------------
 .../parser/dialect/JdbcMetadataDialect.java     | 95 ++++++++++----------
 1 file changed, 46 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49af1cf2/modules/schema-load/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
index 38b095a..ab65e7a 100644
--- a/modules/schema-load/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
@@ -32,11 +32,11 @@ public class JdbcMetadataDialect extends DatabaseMetadataDialect {
     /** */
     private static final String[] TABLES_AND_VIEWS = {"TABLE", "VIEW"};
 
-    /** Schema name index. */
-    private static final int SCHEMA_NAME_IDX = 1;
-
     /** Schema catalog index. */
-    private static final int SCHEMA_CATALOG_IDX = 2;
+    private static final int TBL_CATALOG_IDX = 1;
+
+    /** Schema name index. */
+    private static final int TBL_SCHEMA_IDX = 2;
 
     /** Table name index. */
     private static final int TBL_NAME_IDX = 3;
@@ -70,72 +70,69 @@ public class JdbcMetadataDialect extends DatabaseMetadataDialect {
 
         Collection<DbTable> tbls = new ArrayList<>();
 
-        try (ResultSet schemasRs = dbMeta.getSchemas()) {
-            while (schemasRs.next()) {
-                String schema = schemasRs.getString(SCHEMA_NAME_IDX);
+        try (ResultSet tblsRs = dbMeta.getTables(null, null, "%",
+            tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
+            while (tblsRs.next()) {
+                String tblCatalog = tblsRs.getString(TBL_CATALOG_IDX);
+                String tblSchema = tblsRs.getString(TBL_SCHEMA_IDX);
+                String tblName = tblsRs.getString(TBL_NAME_IDX);
+
+                // In case of MySql we should use catalog.
+                String schema = tblSchema != null ? tblSchema : tblCatalog;
 
                 // Skip system schemas.
                 if (sys.contains(schema))
                     continue;
 
-                String catalog = schemasRs.getString(SCHEMA_CATALOG_IDX);
-
-                try (ResultSet tblsRs = dbMeta.getTables(catalog, schema, "%",
-                    tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
-                    while (tblsRs.next()) {
-                        String tblName = tblsRs.getString(TBL_NAME_IDX);
-
-                        Set<String> pkCols = new HashSet<>();
-
-                        try (ResultSet pkRs = dbMeta.getPrimaryKeys(catalog, schema, tblName)) {
-                            while (pkRs.next())
-                                pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
-                        }
+                Set<String> pkCols = new HashSet<>();
 
-                        List<DbColumn> cols = new ArrayList<>();
+                try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) {
+                    while (pkRs.next())
+                        pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
+                }
 
-                        try (ResultSet colsRs = dbMeta.getColumns(catalog, schema, tblName, null)) {
-                            while (colsRs.next()) {
-                                String colName = colsRs.getString(COL_NAME_IDX);
+                List<DbColumn> cols = new ArrayList<>();
 
-                                cols.add(new DbColumn(
-                                    colName,
-                                    colsRs.getInt(COL_DATA_TYPE_IDX),
-                                    pkCols.contains(colName),
-                                    colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
-                            }
-                        }
+                try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) {
+                    while (colsRs.next()) {
+                        String colName = colsRs.getString(COL_NAME_IDX);
 
-                        Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
+                        cols.add(new DbColumn(
+                            colName,
+                            colsRs.getInt(COL_DATA_TYPE_IDX),
+                            pkCols.contains(colName),
+                            colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
+                    }
+                }
 
-                        try (ResultSet idxRs = dbMeta.getIndexInfo(catalog, schema, tblName, false, true)) {
-                            while (idxRs.next()) {
-                                String idxName = idxRs.getString(IDX_NAME_IDX);
+                Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
 
-                                String colName = idxRs.getString(IDX_COL_NAME_IDX);
+                try (ResultSet idxRs = dbMeta.getIndexInfo(tblCatalog, tblSchema, tblName, false, true)) {
+                    while (idxRs.next()) {
+                        String idxName = idxRs.getString(IDX_NAME_IDX);
 
-                                if (idxName == null || colName == null)
-                                    continue;
+                        String colName = idxRs.getString(IDX_COL_NAME_IDX);
 
-                                Map<String, Boolean> idx = idxs.get(idxName);
+                        if (idxName == null || colName == null)
+                            continue;
 
-                                if (idx == null) {
-                                    idx = new LinkedHashMap<>();
+                        Map<String, Boolean> idx = idxs.get(idxName);
 
-                                    idxs.put(idxName, idx);
-                                }
+                        if (idx == null) {
+                            idx = new LinkedHashMap<>();
 
-                                String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
+                            idxs.put(idxName, idx);
+                        }
 
-                                Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
+                        String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
 
-                                idx.put(colName, desc);
-                            }
-                        }
+                        Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
 
-                        tbls.add(table(schema, tblName, cols, idxs));
+                        idx.put(colName, desc);
                     }
                 }
+
+                tbls.add(table(schema, tblName, cols, idxs));
             }
         }
 


[53/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java


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

Branch: refs/heads/ignite-313
Commit: cce63fdba2913977de6477a9e5a6a663e3ccc3fd
Parents: 17ac360
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Mar 5 11:16:59 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Mar 5 11:16:59 2015 +0300

----------------------------------------------------------------------
 .../external/HadoopExternalTaskExecutor.java         | 15 ++-------------
 1 file changed, 2 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cce63fdb/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 4b749f3..7fe9d19 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
@@ -330,7 +330,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
 
         HadoopJobId jobId = job.id();
 
-        final HadoopProcessFuture fut = new HadoopProcessFuture(childProcId, jobId, ctx.kernalContext());
+        final HadoopProcessFuture fut = new HadoopProcessFuture(childProcId, jobId);
 
         final HadoopProcess proc = new HadoopProcess(jobId, fut, plan.reducers(ctx.localNodeId()));
 
@@ -853,9 +853,6 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      *
      */
     private class HadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, HadoopProcessDescriptor>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
         /** Child process ID. */
         private UUID childProcId;
 
@@ -878,16 +875,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         private final IgniteLogger log = HadoopExternalTaskExecutor.this.log;
 
         /**
-         * Empty constructor.
-         */
-        public HadoopProcessFuture() {
-            // No-op.
-        }
-
-        /**
-         * @param ctx Kernal context.
          */
-        private GridHadoopProcessFuture(UUID childProcId, HadoopJobId jobId, GridKernalContext ctx) {
+        private HadoopProcessFuture(UUID childProcId, HadoopJobId jobId) {
             this.childProcId = childProcId;
             this.jobId = jobId;
         }


[48/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into sprint-2

Posted by yz...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into sprint-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/5d06523b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/5d06523b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/5d06523b

Branch: refs/heads/ignite-313
Commit: 5d06523b38fdea164cd4f1e62759b0daa4c12ce6
Parents: b83700e 42e9485
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Mar 5 10:59:43 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu Mar 5 10:59:43 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       | 61 +++++++++++++++++++-
 1 file changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[58/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1


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

Branch: refs/heads/ignite-313
Commit: 401835cc63fcf7f4412397eb953d41a2599a66c4
Parents: 10bb0da ae413fb
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Mar 5 12:04:41 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Mar 5 12:04:41 2015 +0300

----------------------------------------------------------------------
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  4 +-
 .../parser/dialect/JdbcMetadataDialect.java     | 95 ++++++++++----------
 2 files changed, 49 insertions(+), 50 deletions(-)
----------------------------------------------------------------------



[47/58] [abbrv] incubator-ignite git commit: Minor utility method add.

Posted by yz...@apache.org.
Minor utility method add.


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

Branch: refs/heads/ignite-313
Commit: 42e94854f1faa3c87a18c90b57f7064eb07cc26e
Parents: 6423cf0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Mar 5 10:59:05 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 5 10:59:05 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       | 61 +++++++++++++++++++-
 1 file changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/42e94854/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index de71f59..6b12554 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -7431,6 +7431,66 @@ public abstract class IgniteUtils {
      * @param cls Object.
      * @param obj Object.
      * @param mtdName Field name.
+     * @param params Parameters.
+     * @return Field value.
+     * @throws IgniteCheckedException If static field with given name cannot be retreived.
+     */
+    public static <T> T invoke(@Nullable Class<?> cls, @Nullable Object obj, String mtdName,
+        Object... params) throws IgniteCheckedException {
+        assert cls != null || obj != null;
+        assert mtdName != null;
+
+        try {
+            for (Class<?> c = cls != null ? cls : obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) {
+                Method[] mtds = c.getDeclaredMethods();
+
+                Method mtd = null;
+
+                for (Method declaredMtd : c.getDeclaredMethods()) {
+                    if (declaredMtd.getName().equals(mtdName)) {
+                        if (mtd == null)
+                            mtd = declaredMtd;
+                        else
+                            throw new IgniteCheckedException("Failed to invoke (ambigous method name) [mtdName=" +
+                                mtdName + ", cls=" + cls + ']');
+                    }
+                }
+
+                if (mtd == null)
+                    continue;
+
+                boolean accessible = mtd.isAccessible();
+
+                T res;
+
+                try {
+                    mtd.setAccessible(true);
+
+                    res = (T)mtd.invoke(obj, params);
+                }
+                finally {
+                    if (!accessible)
+                        mtd.setAccessible(false);
+                }
+
+                return res;
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to invoke [mtdName=" + mtdName + ", cls=" + cls + ']',
+                e);
+        }
+
+        throw new IgniteCheckedException("Failed to invoke (method was not found) [mtdName=" + mtdName +
+            ", cls=" + cls + ']');
+    }
+
+    /**
+     * Invokes method.
+     *
+     * @param cls Object.
+     * @param obj Object.
+     * @param mtdName Field name.
      * @param paramTypes Parameter types.
      * @param params Parameters.
      * @return Field value.
@@ -7478,7 +7538,6 @@ public abstract class IgniteUtils {
             ", cls=" + cls + ']');
     }
 
-
     /**
      * Gets property value.
      *


[08/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
new file mode 100644
index 0000000..3b4c5c2
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.typedef.*;
+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 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.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.processors.hadoop.igfs.HadoopIgfsUtils.*;
+import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
+
+/**
+ * Tests for IGFS file system handshake.
+ */
+public class IgniteHadoopFileSystemHandshakeSelfTest extends IgfsCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Grid name. */
+    private static final String GRID_NAME = "grid";
+
+    /** IGFS name. */
+    private static final String IGFS_NAME = "igfs";
+
+    /** IGFS path. */
+    private static final IgfsPath PATH = new IgfsPath("/path");
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+    }
+
+    /**
+     * Tests for Grid and IGFS having normal names.
+     *
+     * @throws Exception If failed.
+     */
+    public void testHandshake() throws Exception {
+        startUp(false, false);
+
+        checkValid(IGFS_NAME + ":" + GRID_NAME + "@");
+        checkValid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
+        checkValid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(IGFS_NAME + "@");
+        checkInvalid(IGFS_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(":" + GRID_NAME + "@");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid("");
+        checkInvalid("127.0.0.1");
+        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
+    }
+
+    /**
+     * Tests for Grid having {@code null} name and IGFS having normal name.
+     *
+     * @throws Exception If failed.
+     */
+    public void testHandshakeDefaultGrid() throws Exception {
+        startUp(true, false);
+
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkValid(IGFS_NAME + "@");
+        checkValid(IGFS_NAME + "@127.0.0.1");
+        checkValid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(":" + GRID_NAME + "@");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid("");
+        checkInvalid("127.0.0.1");
+        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
+    }
+
+    /**
+     * Tests for Grid having normal name and IGFS having {@code null} name.
+     *
+     * @throws Exception If failed.
+     */
+    public void testHandshakeDefaultIgfs() throws Exception {
+        startUp(false, true);
+
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(IGFS_NAME + "@");
+        checkInvalid(IGFS_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkValid(":" + GRID_NAME + "@");
+        checkValid(":" + GRID_NAME + "@127.0.0.1");
+        checkValid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid("");
+        checkInvalid("127.0.0.1");
+        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
+    }
+
+    /**
+     * Tests for Grid having {@code null} name and IGFS having {@code null} name.
+     *
+     * @throws Exception If failed.
+     */
+    public void testHandshakeDefaultGridDefaultIgfs() throws Exception {
+        startUp(true, true);
+
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(IGFS_NAME + "@");
+        checkInvalid(IGFS_NAME + "@127.0.0.1");
+        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkInvalid(":" + GRID_NAME + "@");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
+        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
+
+        checkValid("");
+        checkValid("127.0.0.1");
+        checkValid("127.0.0.1:" + DFLT_IPC_PORT);
+    }
+
+    /**
+     * Perform startup.
+     *
+     * @param dfltGridName Default Grid name.
+     * @param dfltIgfsName Default IGFS name.
+     * @throws Exception If failed.
+     */
+    private void startUp(boolean dfltGridName, boolean dfltIgfsName) throws Exception {
+        Ignite ignite = G.start(gridConfiguration(dfltGridName, dfltIgfsName));
+
+        IgniteFileSystem igfs = ignite.fileSystem(dfltIgfsName ? null : IGFS_NAME);
+
+        igfs.mkdirs(PATH);
+    }
+
+    /**
+     * Create Grid configuration.
+     *
+     * @param dfltGridName Default Grid name.
+     * @param dfltIgfsName Default IGFS name.
+     * @return Grid configuration.
+     * @throws Exception If failed.
+     */
+    private IgniteConfiguration gridConfiguration(boolean dfltGridName, boolean dfltIgfsName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(dfltGridName ? null : GRID_NAME);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("replicated");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        metaCacheCfg.setQueryIndexEnabled(false);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("partitioned");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setDistributionMode(PARTITIONED_ONLY);
+        dataCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setQueryIndexEnabled(false);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        cfg.setCacheConfiguration(metaCacheCfg, dataCacheCfg);
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName(dfltIgfsName ? null : IGFS_NAME);
+        igfsCfg.setPrefetchBlocks(1);
+        igfsCfg.setDefaultMode(PRIMARY);
+        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
+            put("type", "tcp");
+            put("port", String.valueOf(DFLT_IPC_PORT));
+        }});
+
+        igfsCfg.setManagementPort(-1);
+        igfsCfg.setBlockSize(512 * 1024);
+
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Check valid file system endpoint.
+     *
+     * @param authority Authority.
+     * @throws Exception If failed.
+     */
+    private void checkValid(String authority) throws Exception {
+        FileSystem fs = fileSystem(authority);
+
+        assert fs.exists(new Path(PATH.toString()));
+    }
+
+    /**
+     * Check invalid file system endpoint.
+     *
+     * @param authority Authority.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    private void checkInvalid(final String authority) throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fileSystem(authority);
+
+                return null;
+            }
+        }, IOException.class, null);
+    }
+
+    /**
+     *
+     *
+     * @param authority Authority.
+     * @return File system.
+     * @throws Exception If failed.
+     */
+    private static FileSystem fileSystem(String authority) throws Exception {
+        return FileSystem.get(new URI("igfs://" + authority + "/"), configuration(authority));
+    }
+
+    /**
+     * Create configuration for test.
+     *
+     * @param authority Authority.
+     * @return Configuration.
+     */
+    private static Configuration configuration(String authority) {
+        Configuration cfg = new Configuration();
+
+        cfg.set("fs.defaultFS", "igfs://" + authority + "/");
+        cfg.set("fs.igfs.impl", org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.class.getName());
+        cfg.set("fs.AbstractFileSystem.igfs.impl",
+            IgniteHadoopFileSystem.class.getName());
+
+        cfg.setBoolean("fs.igfs.impl.disable.cache", true);
+
+        cfg.setBoolean(String.format(PARAM_IGFS_ENDPOINT_NO_EMBED, authority), true);
+        cfg.setBoolean(String.format(PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority), true);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
new file mode 100644
index 0000000..135a488
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.hadoop.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+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 java.lang.reflect.*;
+import java.net.*;
+import java.util.*;
+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.*;
+
+/**
+ * IPC cache test.
+ */
+public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Path to test hadoop configuration. */
+    private static final String HADOOP_FS_CFG = "modules/core/src/test/config/hadoop/core-site.xml";
+
+    /** Group size. */
+    public static final int GRP_SIZE = 128;
+
+    /** Started grid counter. */
+    private static int cnt;
+
+    /** {@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);
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs");
+        igfsCfg.setManagementPort(FileSystemConfiguration.DFLT_MGMT_PORT + cnt);
+
+        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
+            put("type", "shmem");
+            put("port", String.valueOf(IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT + cnt));
+        }});
+
+        igfsCfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
+
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setCacheConfiguration(cacheConfiguration());
+
+        cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cnt++;
+
+        return cfg;
+    }
+
+    /**
+     * Gets cache configuration.
+     *
+     * @return Cache configuration.
+     */
+    private CacheConfiguration[] cacheConfiguration() {
+        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};
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        G.stopAll(true);
+    }
+
+    /**
+     * Test how IPC cache map works.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testIpcCache() throws Exception {
+        Field cacheField = HadoopIgfsIpcIo.class.getDeclaredField("ipcCache");
+
+        cacheField.setAccessible(true);
+
+        Field activeCntField = HadoopIgfsIpcIo.class.getDeclaredField("activeCnt");
+
+        activeCntField.setAccessible(true);
+
+        Map<String, HadoopIgfsIpcIo> cache = (Map<String, HadoopIgfsIpcIo>)cacheField.get(null);
+
+        String name = "igfs:" + getTestGridName(0) + "@";
+
+        Configuration cfg = new Configuration();
+
+        cfg.addResource(U.resolveIgniteUrl(HADOOP_FS_CFG));
+        cfg.setBoolean("fs.igfs.impl.disable.cache", 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());
+
+        HadoopIgfsIpcIo io = null;
+
+        System.out.println("CACHE: " + cache);
+
+        for (String key : cache.keySet()) {
+            if (key.contains("10500")) {
+                io = cache.get(key);
+
+                break;
+            }
+        }
+
+        assert io != null;
+
+        assertEquals(1, ((AtomicInteger)activeCntField.get(io)).get());
+
+        // Ensure that when IO is used by multiple file systems and one of them is closed, IO is not stopped.
+        FileSystem fs2 = FileSystem.get(new URI("igfs://" + name + "/abc"), cfg);
+
+        assertEquals(1, cache.size());
+        assertEquals(2, ((AtomicInteger)activeCntField.get(io)).get());
+
+        fs2.close();
+
+        assertEquals(1, cache.size());
+        assertEquals(1, ((AtomicInteger)activeCntField.get(io)).get());
+
+        Field stopField = HadoopIgfsIpcIo.class.getDeclaredField("stopping");
+
+        stopField.setAccessible(true);
+
+        assert !(Boolean)stopField.get(io);
+
+        // Ensure that IO is stopped when nobody else is need it.
+        fs1.close();
+
+        assert cache.isEmpty();
+
+        assert (Boolean)stopField.get(io);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
new file mode 100644
index 0000000..8d43e08
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs;
+
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.igfs.common.IgfsLogger.*;
+
+/**
+ * Grid IGFS client logger test.
+ */
+public class IgniteHadoopFileSystemLoggerSelfTest extends IgfsCommonAbstractTest {
+    /** Path string. */
+    private static final String PATH_STR = "/dir1/dir2/file;test";
+
+    /** Path string with escaped semicolons. */
+    private static final String PATH_STR_ESCAPED = PATH_STR.replace(';', '~');
+
+    /** Path. */
+    private static final IgfsPath PATH = new IgfsPath(PATH_STR);
+
+    /** IGFS name. */
+    private static final String IGFS_NAME = "igfs";
+
+    /** Log file path. */
+    private static final String LOG_DIR = U.getIgniteHome();
+
+    /** Endpoint address. */
+    private static final String ENDPOINT = "localhost:10500";
+
+    /** Log file name. */
+    private static final String LOG_FILE = LOG_DIR + File.separator + "igfs-log-" + IGFS_NAME + "-" + U.jvmPid() +
+        ".csv";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        removeLogs();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        removeLogs();
+    }
+
+    /**
+     * Remove existing logs.
+     *
+     * @throws Exception If failed.
+     */
+    private void removeLogs() throws Exception {
+        File dir = new File(LOG_DIR);
+
+        File[] logs = dir.listFiles(new FilenameFilter() {
+            @Override public boolean accept(File dir, String name) {
+                return name.startsWith("igfs-log-");
+            }
+        });
+
+        for (File log : logs)
+            log.delete();
+    }
+
+    /**
+     * Ensure correct static loggers creation/removal as well as file creation.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCreateDelete() throws Exception {
+        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        IgfsLogger sameLog0 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        // Loggers for the same endpoint must be the same object.
+        assert log == sameLog0;
+
+        IgfsLogger otherLog = IgfsLogger.logger("other" + ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        // Logger for another endpoint must be different.
+        assert log != otherLog;
+
+        otherLog.close();
+
+        log.logDelete(PATH, PRIMARY, false);
+
+        log.close();
+
+        File logFile = new File(LOG_FILE);
+
+        // When there are multiple loggers, closing one must not force flushing.
+        assert !logFile.exists();
+
+        IgfsLogger sameLog1 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        assert sameLog0 == sameLog1;
+
+        sameLog0.close();
+
+        assert !logFile.exists();
+
+        sameLog1.close();
+
+        // When we cloe the last logger, it must flush data to disk.
+        assert logFile.exists();
+
+        logFile.delete();
+
+        IgfsLogger sameLog2 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        // This time we expect new logger instance to be created.
+        assert sameLog0 != sameLog2;
+
+        sameLog2.close();
+
+        // As we do not add any records to the logger, we do not expect flushing.
+        assert !logFile.exists();
+    }
+
+    /**
+     * Test read operations logging.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLogRead() throws Exception {
+        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        log.logOpen(1, PATH, PRIMARY, 2, 3L);
+        log.logRandomRead(1, 4L, 5);
+        log.logSeek(1, 6L);
+        log.logSkip(1, 7L);
+        log.logMark(1, 8L);
+        log.logReset(1);
+        log.logCloseIn(1, 9L, 10L, 11);
+
+        log.close();
+
+        checkLog(
+            new SB().a(U.jvmPid() + d() + TYPE_OPEN_IN + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 1 + d() + 2 +
+                d() + 3 + d(14)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_RANDOM_READ + d(3) + 1 + d(7) + 4 + d() + 5 + d(8)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_SEEK + d(3) + 1 + d(7) + 6 + d(9)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_SKIP + d(3) + 1 + d(9) + 7 + d(7)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_MARK + d(3) + 1 + d(10) + 8 + d(6)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_RESET + d(3) + 1 + d(16)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_CLOSE_IN + d(3) + 1 + d(11) + 9 + d() + 10 + d() + 11 + d(3)).toString()
+        );
+    }
+
+    /**
+     * Test write operations logging.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLogWrite() throws Exception {
+        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        log.logCreate(1, PATH, PRIMARY, true, 2, new Integer(3).shortValue(), 4L);
+        log.logAppend(2, PATH, PRIMARY, 8);
+        log.logCloseOut(2, 9L, 10L, 11);
+
+        log.close();
+
+        checkLog(
+            new SB().a(U.jvmPid() + d() + TYPE_OPEN_OUT + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 1 + d() +
+                2 + d(2) + 0 + d() + 1 + d() + 3 + d() + 4 + d(10)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_OPEN_OUT + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 2 + d() +
+                8 + d(2) + 1 + d(13)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_CLOSE_OUT + d(3) + 2 + d(11) + 9 + d() + 10 + d() + 11 + d(3))
+                .toString()
+        );
+    }
+
+    /**
+     * Test miscellaneous operations logging.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testLogMisc() throws Exception {
+        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
+
+        String newFile = "/dir3/file.test";
+        String file1 = "/dir3/file1.test";
+        String file2 = "/dir3/file1.test";
+
+        log.logMakeDirectory(PATH, PRIMARY);
+        log.logRename(PATH, PRIMARY, new IgfsPath(newFile));
+        log.logListDirectory(PATH, PRIMARY, new String[] { file1, file2 });
+        log.logDelete(PATH, PRIMARY, false);
+
+        log.close();
+
+        checkLog(
+            new SB().a(U.jvmPid() + d() + TYPE_DIR_MAKE + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(17)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_RENAME + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(15) + newFile +
+                d(2)).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_DIR_LIST + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(17) + file1 +
+                DELIM_FIELD_VAL + file2).toString(),
+            new SB().a(U.jvmPid() + d() + TYPE_DELETE + d(1) + PATH_STR_ESCAPED + d() + PRIMARY + d(16) + 0 +
+                d()).toString()
+        );
+    }
+
+    /**
+     * Create IGFS file with the given path.
+     *
+     * @param path File path.
+     * @return IGFS file instance.
+     */
+    private IgfsFile file(String path) {
+        return new IgfsFileImpl(new IgfsPath(path), new IgfsFileInfo(), 64 * 1024 * 1024);
+    }
+
+    /**
+     * Ensure that log file has only the following lines.
+     *
+     * @param lines Expected lines.
+     */
+    private void checkLog(String... lines) throws Exception {
+        BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(LOG_FILE)));
+
+        List<String> logLines = new ArrayList<>(lines.length);
+
+        String nextLogLine;
+
+        while ((nextLogLine = br.readLine()) != null)
+            logLines.add(nextLogLine);
+
+        U.closeQuiet(br);
+
+        assertEquals(lines.length + 1, logLines.size());
+
+        assertEquals(logLines.get(0), HDR);
+
+        for (int i = 0; i < lines.length; i++) {
+            String logLine = logLines.get(i + 1);
+
+            logLine = logLine.substring(logLine.indexOf(DELIM_FIELD, logLine.indexOf(DELIM_FIELD) + 1) + 1);
+
+            assertEquals(lines[i], logLine);
+        }
+    }
+
+    /**
+     * Return single field delimiter.
+     *
+     * @return Single field delimiter.
+     */
+    private String d() {
+        return d(1);
+    }
+
+    /**
+     * Return a bunch of field delimiters.
+     *
+     * @param cnt Amount of field delimiters.
+     * @return Field delimiters.
+     */
+    private String d(int cnt) {
+        SB buf = new SB();
+
+        for (int i = 0; i < cnt; i++)
+            buf.a(DELIM_FIELD);
+
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java
new file mode 100644
index 0000000..1f6a204
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.FileSystem;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+import org.apache.ignite.internal.igfs.common.*;
+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 java.lang.reflect.*;
+import java.net.*;
+import java.nio.file.*;
+import java.util.*;
+
+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.*;
+
+/**
+ * Ensures that sampling is really turned on/off.
+ */
+public class IgniteHadoopFileSystemLoggerStateSelfTest extends IgfsCommonAbstractTest {
+    /** IGFS. */
+    private IgfsEx igfs;
+
+    /** File system. */
+    private FileSystem fs;
+
+    /** Whether logging is enabled in FS configuration. */
+    private boolean logging;
+
+    /** whether sampling is enabled. */
+    private Boolean sampling;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        U.closeQuiet(fs);
+
+        igfs = null;
+        fs = null;
+
+        G.stopAll(true);
+
+        logging = false;
+        sampling = null;
+    }
+
+    /**
+     * Startup the grid and instantiate the file system.
+     *
+     * @throws Exception If failed.
+     */
+    private void startUp() throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs");
+        igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setDefaultMode(PRIMARY);
+        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
+            put("type", "tcp");
+            put("port", "10500");
+        }});
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        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("igfs-grid");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        Ignite g = G.start(cfg);
+
+        igfs = (IgfsEx)g.fileSystem("igfs");
+
+        igfs.globalSampling(sampling);
+
+        fs = fileSystem();
+    }
+
+    /**
+     * When logging is disabled and sampling is not set no-op logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingDisabledSamplingNotSet() throws Exception {
+        startUp();
+
+        assert !logEnabled();
+    }
+
+    /**
+     * When logging is enabled and sampling is not set file logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingEnabledSamplingNotSet() throws Exception {
+        logging = true;
+
+        startUp();
+
+        assert logEnabled();
+    }
+
+    /**
+     * When logging is disabled and sampling is disabled no-op logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingDisabledSamplingDisabled() throws Exception {
+        sampling = false;
+
+        startUp();
+
+        assert !logEnabled();
+    }
+
+    /**
+     * When logging is enabled and sampling is disabled no-op logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingEnabledSamplingDisabled() throws Exception {
+        logging = true;
+        sampling = false;
+
+        startUp();
+
+        assert !logEnabled();
+    }
+
+    /**
+     * When logging is disabled and sampling is enabled file logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingDisabledSamplingEnabled() throws Exception {
+        sampling = true;
+
+        startUp();
+
+        assert logEnabled();
+    }
+
+    /**
+     * When logging is enabled and sampling is enabled file logger must be used.
+     *
+     * @throws Exception If failed.
+     */
+    public void testLoggingEnabledSamplingEnabled() throws Exception {
+        logging = true;
+        sampling = true;
+
+        startUp();
+
+        assert logEnabled();
+    }
+
+    /**
+     * Ensure sampling change through API causes changes in logging on subsequent client connections.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSamplingChange() throws Exception {
+        // Start with sampling not set.
+        startUp();
+
+        assert !logEnabled();
+
+        fs.close();
+
+        // "Not set" => true transition.
+        igfs.globalSampling(true);
+
+        fs = fileSystem();
+
+        assert logEnabled();
+
+        fs.close();
+
+        // True => "not set" transition.
+        igfs.globalSampling(null);
+
+        fs = fileSystem();
+
+        assert !logEnabled();
+
+        // "Not-set" => false transition.
+        igfs.globalSampling(false);
+
+        fs = fileSystem();
+
+        assert !logEnabled();
+
+        fs.close();
+
+        // False => "not=set" transition.
+        igfs.globalSampling(null);
+
+        fs = fileSystem();
+
+        assert !logEnabled();
+
+        fs.close();
+
+        // True => false transition.
+        igfs.globalSampling(true);
+        igfs.globalSampling(false);
+
+        fs = fileSystem();
+
+        assert !logEnabled();
+
+        fs.close();
+
+        // False => true transition.
+        igfs.globalSampling(true);
+
+        fs = fileSystem();
+
+        assert logEnabled();
+    }
+
+    /**
+     * Ensure that log directory is set to IGFS when client FS connects.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testLogDirectory() throws Exception {
+        startUp();
+
+        assertEquals(Paths.get(U.getIgniteHome()).normalize().toString(),
+            igfs.clientLogDirectory());
+    }
+
+    /**
+     * Instantiate new file system.
+     *
+     * @return New file system.
+     * @throws Exception If failed.
+     */
+    private IgniteHadoopFileSystem fileSystem() throws Exception {
+        Configuration fsCfg = new Configuration();
+
+        fsCfg.addResource(U.resolveIgniteUrl("modules/core/src/test/config/hadoop/core-site-loopback.xml"));
+
+        fsCfg.setBoolean("fs.igfs.impl.disable.cache", true);
+
+        if (logging)
+            fsCfg.setBoolean(String.format(PARAM_IGFS_LOG_ENABLED, "igfs:igfs-grid@"), logging);
+
+        fsCfg.setStrings(String.format(PARAM_IGFS_LOG_DIR, "igfs:igfs-grid@"), U.getIgniteHome());
+
+        return (IgniteHadoopFileSystem)FileSystem.get(new URI("igfs://igfs:igfs-grid@/"), fsCfg);
+    }
+
+    /**
+     * Ensure that real logger is used by the file system.
+     *
+     * @return {@code True} in case path is secondary.
+     * @throws Exception If failed.
+     */
+    private boolean logEnabled() throws Exception {
+        assert fs != null;
+
+        Field field = fs.getClass().getDeclaredField("clientLog");
+
+        field.setAccessible(true);
+
+        return ((IgfsLogger)field.get(fs)).isLogEnabled();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java
new file mode 100644
index 0000000..e33cde7
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.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.igfs;
+
+import java.util.*;
+
+import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test.
+ */
+public abstract class IgniteHadoopFileSystemLoopbackAbstractSelfTest extends
+    IgniteHadoopFileSystemAbstractSelfTest {
+    /**
+     * Constructor.
+     *
+     * @param mode IGFS mode.
+     * @param skipEmbed Skip embedded mode flag.
+     */
+    protected IgniteHadoopFileSystemLoopbackAbstractSelfTest(IgfsMode mode, boolean skipEmbed) {
+        super(mode, skipEmbed, true);
+    }
+
+    /** {@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)));
+        }};
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
new file mode 100644
index 0000000..d46001d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in DUAL_ASYNC mode.
+ */
+public class IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest extends
+    IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest() {
+        super(DUAL_ASYNC, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
new file mode 100644
index 0000000..fa64734
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in DUAL_SYNC mode.
+ */
+public class IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest
+    extends IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest() {
+        super(DUAL_SYNC, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
new file mode 100644
index 0000000..9035acd
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in PRIMARY mode.
+ */
+public class IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest
+    extends IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest() {
+        super(PRIMARY, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
new file mode 100644
index 0000000..8198c52
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in SECONDARY mode.
+ */
+public class IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest extends
+    IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest() {
+        super(PROXY, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
new file mode 100644
index 0000000..246c516
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in DUAL_ASYNC mode.
+ */
+public class IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest extends
+    IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest() {
+        super(DUAL_ASYNC, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
new file mode 100644
index 0000000..b5140af
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in DUAL_SYNC mode.
+ */
+public class IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest
+    extends IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest() {
+        super(DUAL_SYNC, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java
new file mode 100644
index 0000000..572ac4b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in PRIMARY mode.
+ */
+public class IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest
+    extends IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest() {
+        super(PRIMARY, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java
new file mode 100644
index 0000000..006c271
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC loopback self test in SECONDARY mode.
+ */
+public class IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest extends
+    IgniteHadoopFileSystemLoopbackAbstractSelfTest {
+
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest() {
+        super(PROXY, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryModeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryModeSelfTest.java
new file mode 100644
index 0000000..3f20070
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryModeSelfTest.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+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.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+
+import java.net.*;
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+
+/**
+ * Ensures correct modes resolution for SECONDARY paths.
+ */
+public class IgniteHadoopFileSystemSecondaryModeSelfTest extends IgfsCommonAbstractTest {
+    /** Path to check. */
+    private static final Path PATH = new Path("/dir");
+
+    /** Pattern matching the path. */
+    private static final String PATTERN_MATCHES = "/dir";
+
+    /** Pattern doesn't matching the path. */
+    private static final String PATTERN_NOT_MATCHES = "/files";
+
+    /** Default IGFS mode. */
+    private IgfsMode mode;
+
+    /** Path modes. */
+    private Map<String, IgfsMode> pathModes;
+
+    /** File system. */
+    private IgniteHadoopFileSystem fs;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        mode = null;
+        pathModes = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        U.closeQuiet(fs);
+
+        fs = null;
+
+        G.stopAll(true);
+    }
+
+    /**
+     * Perform initial startup.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("NullableProblems")
+    private void startUp() throws Exception {
+        startUpSecondary();
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs");
+        igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setPathModes(pathModes);
+        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
+            put("type", "tcp");
+            put("port", "10500");
+        }});
+
+        igfsCfg.setManagementPort(-1);
+        igfsCfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
+            "igfs://igfs-secondary:igfs-grid-secondary@127.0.0.1:11500/",
+            "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml"));
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        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("igfs-grid");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        G.start(cfg);
+
+        Configuration fsCfg = new Configuration();
+
+        fsCfg.addResource(U.resolveIgniteUrl("modules/core/src/test/config/hadoop/core-site-loopback.xml"));
+
+        fsCfg.setBoolean("fs.igfs.impl.disable.cache", true);
+
+        fs = (IgniteHadoopFileSystem)FileSystem.get(new URI("igfs://igfs:igfs-grid@/"), fsCfg);
+    }
+
+    /**
+     * Startup secondary file system.
+     *
+     * @throws Exception If failed.
+     */
+    private void startUpSecondary() throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("partitioned");
+        igfsCfg.setMetaCacheName("replicated");
+        igfsCfg.setName("igfs-secondary");
+        igfsCfg.setBlockSize(512 * 1024);
+        igfsCfg.setDefaultMode(PRIMARY);
+        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
+            put("type", "tcp");
+            put("port", "11500");
+        }});
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        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("igfs-grid-secondary");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        G.start(cfg);
+    }
+
+    /**
+     * Check path resolution when secondary mode is not default and there are no other exclusion paths.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryNotDefaultNoExclusions() throws Exception {
+        mode = PRIMARY;
+
+        startUp();
+
+        assert !secondary(PATH);
+        assert !secondary(PATH);
+    }
+
+    /**
+     * Check path resolution when secondary mode is not default and there is no matching exclusion paths.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryNotDefaultNonMatchingExclusion() throws Exception {
+        mode = PRIMARY;
+
+        pathModes(F.t(PATTERN_NOT_MATCHES, PROXY));
+
+        startUp();
+
+        assert !secondary(PATH);
+        assert !secondary(PATH);
+    }
+
+    /**
+     * Check path resolution when secondary mode is not default and there is matching exclusion path.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryNotDefaultMatchingExclusion() throws Exception {
+        mode = PRIMARY;
+
+        pathModes(F.t(PATTERN_NOT_MATCHES, PROXY), F.t(PATTERN_MATCHES, PROXY));
+
+        startUp();
+
+        assert secondary(PATH);
+        assert secondary(PATH);
+    }
+
+    /**
+     * Check path resolution when secondary mode is default and there is no exclusion paths.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryDefaultNoExclusions() throws Exception {
+        mode = PROXY;
+
+        startUp();
+
+        assert secondary(PATH);
+        assert secondary(PATH);
+    }
+
+    /**
+     * Check path resolution when secondary mode is default and there is no matching exclusion paths.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryDefaultNonMatchingExclusion() throws Exception {
+        mode = PROXY;
+
+        pathModes(F.t(PATTERN_NOT_MATCHES, PRIMARY));
+
+        startUp();
+
+        assert secondary(PATH);
+        assert secondary(PATH);
+    }
+
+    /**
+     * Check path resolution when secondary mode is default and there is no matching exclusion paths.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSecondaryDefaultMatchingExclusion() throws Exception {
+        mode = PROXY;
+
+        pathModes(F.t(PATTERN_NOT_MATCHES, PRIMARY), F.t(PATTERN_MATCHES, PRIMARY));
+
+        startUp();
+
+        assert !secondary(PATH);
+        assert !secondary(PATH);
+    }
+
+    /**
+     * Set IGFS modes for particular paths.
+     *
+     * @param modes Modes.
+     */
+    @SafeVarargs
+    final void pathModes(IgniteBiTuple<String, IgfsMode>... modes) {
+        assert modes != null;
+
+        pathModes = new LinkedHashMap<>(modes.length, 1.0f);
+
+        for (IgniteBiTuple<String, IgfsMode> mode : modes)
+            pathModes.put(mode.getKey(), mode.getValue());
+    }
+
+    /**
+     * Check whether the given path is threaten as SECONDARY in the file system.
+     *
+     * @param path Path to check.
+     * @return {@code True} in case path is secondary.
+     * @throws Exception If failed.
+     */
+    private boolean secondary(Path path) throws Exception {
+        return fs.mode(path) == PROXY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java
new file mode 100644
index 0000000..991045b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.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.igfs;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
+
+/**
+ * IGFS Hadoop file system IPC self test.
+ */
+public abstract class IgniteHadoopFileSystemShmemAbstractSelfTest extends IgniteHadoopFileSystemAbstractSelfTest {
+    /**
+     * Constructor.
+     *
+     * @param mode IGFS mode.
+     * @param skipEmbed Skip embedded mode flag.
+     */
+    protected IgniteHadoopFileSystemShmemAbstractSelfTest(IgfsMode mode, boolean skipEmbed) {
+        super(mode, skipEmbed, false);
+    }
+
+    /** {@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)));
+        }};
+    }
+
+    /**
+     * Checks correct behaviour in case when we run out of system
+     * resources.
+     *
+     * @throws Exception If error occurred.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testOutOfResources() throws Exception {
+        final Collection<IpcEndpoint> eps = new LinkedList<>();
+
+        try {
+            IgniteCheckedException e = (IgniteCheckedException)GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @SuppressWarnings("InfiniteLoopStatement")
+                @Override public Object call() throws Exception {
+                    while (true) {
+                        IpcEndpoint ep = IpcEndpointFactory.connectEndpoint("shmem:10500", log);
+
+                        eps.add(ep);
+                    }
+                }
+            }, IgniteCheckedException.class, null);
+
+            assertNotNull(e);
+
+            String msg = e.getMessage();
+
+            assertTrue("Invalid exception: " + X.getFullStackTrace(e),
+                msg.contains("(error code: 28)") ||
+                msg.contains("(error code: 24)") ||
+                msg.contains("(error code: 12)"));
+        }
+        finally {
+            for (IpcEndpoint ep : eps)
+                ep.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
new file mode 100644
index 0000000..ed34398
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in DUAL_ASYNC mode.
+ */
+public class IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest() {
+        super(DUAL_ASYNC, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
new file mode 100644
index 0000000..4b7bfb1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in DUAL_SYNC mode.
+ */
+public class IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest() {
+        super(DUAL_SYNC, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
new file mode 100644
index 0000000..c1393d3
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in PRIMARY mode.
+ */
+public class IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest() {
+        super(PRIMARY, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
new file mode 100644
index 0000000..2c97e7b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in SECONDARY mode.
+ */
+public class IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest() {
+        super(PROXY, false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
new file mode 100644
index 0000000..6d9973f
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in DUAL_ASYNC mode.
+ */
+public class IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest() {
+        super(DUAL_ASYNC, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java
new file mode 100644
index 0000000..6ec084d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in DUAL_SYNC mode.
+ */
+public class IgniteHadoopFileSystemShmemExternalDualSyncSelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemExternalDualSyncSelfTest() {
+        super(DUAL_SYNC, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java
new file mode 100644
index 0000000..03f3fee
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in PRIMARY mode.
+ */
+public class IgniteHadoopFileSystemShmemExternalPrimarySelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemExternalPrimarySelfTest() {
+        super(PRIMARY, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java
new file mode 100644
index 0000000..bc27674
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;
+
+/**
+ * IGFS Hadoop file system IPC shmem self test in SECONDARY mode.
+ */
+public class IgniteHadoopFileSystemShmemExternalSecondarySelfTest
+    extends IgniteHadoopFileSystemShmemAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgniteHadoopFileSystemShmemExternalSecondarySelfTest() {
+        super(PROXY, true);
+    }
+}


[40/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/GridCacheIgfsPerBlockLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/GridCacheIgfsPerBlockLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/GridCacheIgfsPerBlockLruEvictionPolicySelfTest.java
deleted file mode 100644
index f0e2c70..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/GridCacheIgfsPerBlockLruEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,485 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.igfs;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.igfs.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.util.lang.*;
-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 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.*;
-
-/**
- * Tests for IGFS per-block LR eviction policy.
- */
-@SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
-public class GridCacheIgfsPerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstractTest {
-    /** Primary IGFS name. */
-    private static final String IGFS_PRIMARY = "igfs-primary";
-
-    /** Primary IGFS name. */
-    private static final String IGFS_SECONDARY = "igfs-secondary";
-
-    /** Secondary file system REST endpoint configuration map. */
-    private static final Map<String, String> SECONDARY_REST_CFG = new HashMap<String, String>() {{
-        put("type", "tcp");
-        put("port", "11500");
-    }};
-
-    /** File working in PRIMARY mode. */
-    public static final IgfsPath FILE = new IgfsPath("/file");
-
-    /** File working in DUAL mode. */
-    public static final IgfsPath FILE_RMT = new IgfsPath("/fileRemote");
-
-    /** Primary IGFS instances. */
-    private static IgfsImpl igfsPrimary;
-
-    /** Secondary IGFS instance. */
-    private static IgniteFs secondaryFs;
-
-    /** Primary file system data cache. */
-    private static GridCacheAdapter<IgfsBlockKey, byte[]> dataCache;
-
-    /** Eviction policy */
-    private static CacheIgfsPerBlockLruEvictionPolicy evictPlc;
-
-    /**
-     * Start a grid with the primary file system.
-     *
-     * @throws Exception If failed.
-     */
-    private void startPrimary() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
-        igfsCfg.setName(IGFS_PRIMARY);
-        igfsCfg.setBlockSize(512);
-        igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setPrefetchBlocks(1);
-        igfsCfg.setSequentialReadsBeforePrefetch(Integer.MAX_VALUE);
-        igfsCfg.setSecondaryFileSystem(secondaryFs);
-
-        Map<String, IgfsMode> pathModes = new HashMap<>();
-
-        pathModes.put(FILE_RMT.toString(), DUAL_SYNC);
-
-        igfsCfg.setPathModes(pathModes);
-
-        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
-
-        dataCacheCfg.setName("dataCache");
-        dataCacheCfg.setCacheMode(PARTITIONED);
-        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        evictPlc = new CacheIgfsPerBlockLruEvictionPolicy();
-
-        dataCacheCfg.setEvictionPolicy(evictPlc);
-        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        dataCacheCfg.setBackups(0);
-        dataCacheCfg.setQueryIndexEnabled(false);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("metaCache");
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        metaCacheCfg.setQueryIndexEnabled(false);
-        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        IgniteConfiguration cfg = new IgniteConfiguration();
-
-        cfg.setGridName("grid-primary");
-
-        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);
-
-        Ignite g = G.start(cfg);
-
-        igfsPrimary = (IgfsImpl)g.fileSystem(IGFS_PRIMARY);
-
-        dataCache = igfsPrimary.context().kernalContext().cache().internalCache(
-            igfsPrimary.context().configuration().getDataCacheName());
-    }
-
-    /**
-     * Start a grid with the secondary file system.
-     *
-     * @throws Exception If failed.
-     */
-    private void startSecondary() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
-        igfsCfg.setName(IGFS_SECONDARY);
-        igfsCfg.setBlockSize(512);
-        igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setIpcEndpointConfiguration(SECONDARY_REST_CFG);
-
-        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
-
-        dataCacheCfg.setName("dataCache");
-        dataCacheCfg.setCacheMode(PARTITIONED);
-        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        dataCacheCfg.setBackups(0);
-        dataCacheCfg.setQueryIndexEnabled(false);
-        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("metaCache");
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        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(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setConnectorConfiguration(null);
-
-        Ignite g = G.start(cfg);
-
-        secondaryFs = g.fileSystem(IGFS_SECONDARY);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        try {
-            // Cleanup.
-            igfsPrimary.format();
-
-            while (!dataCache.isEmpty())
-                U.sleep(100);
-
-            checkEvictionPolicy(0, 0);
-        }
-        finally {
-            stopAllGrids(false);
-        }
-    }
-
-    /**
-     * Startup primary and secondary file systems.
-     *
-     * @throws Exception If failed.
-     */
-    private void start() throws Exception {
-        startSecondary();
-        startPrimary();
-
-        evictPlc.setMaxBlocks(0);
-        evictPlc.setMaxSize(0);
-        evictPlc.setExcludePaths(null);
-    }
-
-    /**
-     * Test how evictions are handled for a file working in PRIMARY mode.
-     *
-     * @throws Exception If failed.
-     */
-    public void testFilePrimary() throws Exception {
-        start();
-
-        // Create file in primary mode. It must not be propagated to eviction policy.
-        igfsPrimary.create(FILE, true).close();
-
-        checkEvictionPolicy(0, 0);
-
-        int blockSize = igfsPrimary.info(FILE).blockSize();
-
-        append(FILE, blockSize);
-
-        checkEvictionPolicy(0, 0);
-
-        read(FILE, 0, blockSize);
-
-        checkEvictionPolicy(0, 0);
-    }
-
-    /**
-     * Test how evictions are handled for a file working in PRIMARY mode.
-     *
-     * @throws Exception If failed.
-     */
-    public void testFileDual() throws Exception {
-        start();
-
-        igfsPrimary.create(FILE_RMT, true).close();
-
-        checkEvictionPolicy(0, 0);
-
-        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
-
-        // File write.
-        append(FILE_RMT, blockSize);
-
-        checkEvictionPolicy(1, blockSize);
-
-        // One more write.
-        append(FILE_RMT, blockSize);
-
-        checkEvictionPolicy(2, blockSize * 2);
-
-        // Read.
-        read(FILE_RMT, 0, blockSize);
-
-        checkEvictionPolicy(2, blockSize * 2);
-    }
-
-    /**
-     * Ensure that a DUAL mode file is not propagated to eviction policy
-     *
-     * @throws Exception If failed.
-     */
-    public void testFileDualExclusion() throws Exception {
-        start();
-
-        evictPlc.setExcludePaths(Collections.singleton(FILE_RMT.toString()));
-
-        // Create file in primary mode. It must not be propagated to eviction policy.
-        igfsPrimary.create(FILE_RMT, true).close();
-
-        checkEvictionPolicy(0, 0);
-
-        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
-
-        append(FILE_RMT, blockSize);
-
-        checkEvictionPolicy(0, 0);
-
-        read(FILE_RMT, 0, blockSize);
-
-        checkEvictionPolicy(0, 0);
-    }
-
-    /**
-     * Ensure that exception is thrown in case we are trying to rename file with one exclude setting to the file with
-     * another.
-     *
-     * @throws Exception If failed.
-     */
-    public void testRenameDifferentExcludeSettings() throws Exception {
-        start();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                igfsPrimary.rename(FILE, FILE_RMT);
-
-                return null;
-            }
-        }, IgfsInvalidPathException.class, "Cannot move file to a path with different eviction exclude setting " +
-            "(need to copy and remove)");
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                igfsPrimary.rename(FILE_RMT, FILE);
-
-                return null;
-            }
-        }, IgfsInvalidPathException.class, "Cannot move file to a path with different eviction exclude setting " +
-            "(need to copy and remove)");
-    }
-
-    /**
-     * Test eviction caused by too much blocks.
-     *
-     * @throws Exception If failed.
-     */
-    public void testBlockCountEviction() throws Exception {
-        start();
-
-        int blockCnt = 3;
-
-        evictPlc.setMaxBlocks(blockCnt);
-
-        igfsPrimary.create(FILE_RMT, true).close();
-
-        checkEvictionPolicy(0, 0);
-
-        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
-
-        // Write blocks up to the limit.
-        append(FILE_RMT, blockSize * blockCnt);
-
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Write one more block what should cause eviction.
-        append(FILE_RMT, blockSize);
-
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Read the first block.
-        read(FILE_RMT, 0, blockSize);
-
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-        checkMetrics(1, 1);
-    }
-
-    /**
-     * Test eviction caused by too big data size.
-     *
-     * @throws Exception If failed.
-     */
-    public void testDataSizeEviction() throws Exception {
-        start();
-
-        igfsPrimary.create(FILE_RMT, true).close();
-
-        int blockCnt = 3;
-        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
-
-        evictPlc.setMaxSize(blockSize * blockCnt);
-
-        // Write blocks up to the limit.
-        append(FILE_RMT, blockSize * blockCnt);
-
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Reset metrics.
-        igfsPrimary.resetMetrics();
-
-        // Read the first block what should cause reordering.
-        read(FILE_RMT, 0, blockSize);
-
-        checkMetrics(1, 0);
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Write one more block what should cause eviction of the block 2.
-        append(FILE_RMT, blockSize);
-
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Read the first block.
-        read(FILE_RMT, 0, blockSize);
-
-        checkMetrics(2, 0);
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-
-        // Read the second block (which was evicted).
-        read(FILE_RMT, blockSize, blockSize);
-
-        checkMetrics(3, 1);
-        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
-    }
-
-    /**
-     * Read some data from the given file with the given offset.
-     *
-     * @param path File path.
-     * @param off Offset.
-     * @param len Length.
-     * @throws Exception If failed.
-     */
-    private void read(IgfsPath path, int off, int len) throws Exception {
-        IgfsInputStream is = igfsPrimary.open(path);
-
-        is.readFully(off, new byte[len]);
-
-        is.close();
-    }
-
-    /**
-     * Append some data to the given file.
-     *
-     * @param path File path.
-     * @param len Data length.
-     * @throws Exception If failed.
-     */
-    private void append(IgfsPath path, int len) throws Exception {
-        IgfsOutputStream os = igfsPrimary.append(path, false);
-
-        os.write(new byte[len]);
-
-        os.close();
-    }
-
-    /**
-     * Check metrics counters.
-     *
-     * @param blocksRead Expected blocks read.
-     * @param blocksReadRmt Expected blocks read remote.
-     * @throws Exception If failed.
-     */
-    public void checkMetrics(final long blocksRead, final long blocksReadRmt) throws Exception {
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                IgfsMetrics metrics = igfsPrimary.metrics();
-
-                return metrics.blocksReadTotal() == blocksRead && metrics.blocksReadRemote() == blocksReadRmt;
-            }
-        }, 5000) : "Unexpected metrics [expectedBlocksReadTotal=" + blocksRead + ", actualBlocksReadTotal=" +
-            igfsPrimary.metrics().blocksReadTotal() + ", expectedBlocksReadRemote=" + blocksReadRmt +
-            ", actualBlocksReadRemote=" + igfsPrimary.metrics().blocksReadRemote() + ']';
-    }
-
-    /**
-     * Check eviction policy state.
-     *
-     * @param curBlocks Current blocks.
-     * @param curBytes Current bytes.
-     */
-    private void checkEvictionPolicy(final int curBlocks, final long curBytes) throws IgniteInterruptedCheckedException {
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return evictPlc.getCurrentBlocks() == curBlocks && evictPlc.getCurrentSize() == curBytes;
-            }
-        }, 5000) : "Unexpected counts [expectedBlocks=" + curBlocks + ", actualBlocks=" + evictPlc.getCurrentBlocks() +
-            ", expectedBytes=" + curBytes + ", currentBytes=" + curBytes + ']';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index a7e6780..9a99611 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -37,11 +38,11 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.IgniteFs.*;
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMemoryMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
 
 /**
  * Test fo regular igfs operations.
@@ -154,7 +155,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         igfsSecondary = (IgfsImpl) igniteSecondary.fileSystem("igfs-secondary");
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondary, PRIMARY_REST_CFG);
+        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondary.asSecondary(), PRIMARY_REST_CFG);
 
         igfs = (IgfsImpl) ignite.fileSystem("igfs");
     }
@@ -181,8 +182,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @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();
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable Map<String, String> restCfg) throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
@@ -225,7 +226,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
         cfg.setConnectorConfiguration(null);
@@ -262,7 +263,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testExists() throws Exception {
-        create(igfs, paths(DIR), null);
+        create(igfs.asSecondary(), paths(DIR), null);
 
         checkExist(igfs, igfsSecondary, DIR);
     }
@@ -373,7 +374,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         IgfsPath file1 = new IgfsPath("/file1");
         IgfsPath file2 = new IgfsPath("/file2");
 
-        create(igfs, null, paths(file1));
+        create(igfs.asSecondary(), null, paths(file1));
 
         igfs.rename(file1, file2);
 
@@ -405,7 +406,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         IgfsPath dir1 = new IgfsPath("/dir1");
         IgfsPath dir2 = new IgfsPath("/dir2");
 
-        create(igfs, paths(dir1), null);
+        create(igfs.asSecondary(), paths(dir1), null);
 
         igfs.rename(dir1, dir2);
 
@@ -434,7 +435,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testMoveFileDestinationRoot() throws Exception {
-        create(igfs, paths(DIR, SUBDIR), paths(FILE));
+        create(igfs.asSecondary(), paths(DIR, SUBDIR), paths(FILE));
 
         igfs.rename(FILE, new IgfsPath());
 
@@ -526,7 +527,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testMoveDirectoryDestinationRoot() throws Exception {
-        create(igfs, paths(DIR, SUBDIR, SUBSUBDIR), null);
+        create(igfs.asSecondary(), paths(DIR, SUBDIR, SUBSUBDIR), null);
 
         igfs.rename(SUBSUBDIR, new IgfsPath());
 
@@ -542,7 +543,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testMoveDirectorySourceParentRoot() throws Exception {
         IgfsPath dir = new IgfsPath("/" + SUBSUBDIR.name());
 
-        create(igfs, paths(DIR_NEW, SUBDIR_NEW, dir), null);
+        create(igfs.asSecondary(), paths(DIR_NEW, SUBDIR_NEW, dir), null);
 
         igfs.rename(dir, SUBDIR_NEW);
 
@@ -851,7 +852,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testOpen() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, true, chunk);
+        createFile(igfs.asSecondary(), FILE, true, chunk);
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -886,9 +887,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCreate() throws Exception {
-        create(igfs, paths(DIR, SUBDIR), null);
+        create(igfs.asSecondary(), paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, true, chunk);
+        createFile(igfs.asSecondary(), FILE, true, chunk);
 
         checkFile(igfs, igfsSecondary, FILE, chunk);
     }
@@ -901,7 +902,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testCreateParentRoot() throws Exception {
         IgfsPath file = new IgfsPath("/" + FILE.name());
 
-        createFile(igfs, file, true, chunk);
+        createFile(igfs.asSecondary(), file, true, chunk);
 
         checkFile(igfs, igfsSecondary, file, chunk);
     }
@@ -1092,7 +1093,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                         assert igfs.exists(path);
                     }
 
-                    awaitFileClose(igfs, path);
+                    awaitFileClose(igfs.asSecondary(), path);
 
                     checkFileContent(igfs, path, chunk);
                 }
@@ -1166,7 +1167,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         fut.get();
 
-        awaitFileClose(igfs, FILE);
+        awaitFileClose(igfs.asSecondary(), FILE);
 
         if (err.get() != null)
             throw err.get();
@@ -1212,7 +1213,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testAppendNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         GridTestUtils.assertThrowsInherited(log(), new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -1241,7 +1242,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testAppendRenameNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         IgfsOutputStream os = null;
 
@@ -1263,9 +1264,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAppendRenameParentNoClose() throws Exception {
-        create(igfs, paths(DIR, SUBDIR), null);
+        create(igfs.asSecondary(), paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         IgfsOutputStream os = null;
 
@@ -1289,7 +1290,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testAppendDeleteNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -1321,7 +1322,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testAppendDeleteParentNoClose() throws Exception {
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -1360,7 +1361,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         create(igfs, paths(DIR, SUBDIR), null);
 
-        createFile(igfs, FILE, false);
+        createFile(igfs.asSecondary(), FILE, false);
 
         IgfsOutputStream os = null;
 
@@ -1388,7 +1389,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         int threadCnt = 10;
 
         for (int i = 0; i < threadCnt; i++)
-            createFile(igfs, new IgfsPath("/file" + i), false);
+            createFile(igfs.asSecondary(), new IgfsPath("/file" + i), false);
 
         multithreaded(new Runnable() {
             @Override public void run() {
@@ -1411,7 +1412,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
                         assert igfs.exists(path);
                     }
 
-                    awaitFileClose(igfs, path);
+                    awaitFileClose(igfs.asSecondary(), path);
 
                     checkFileContent(igfs, path, chunks);
                 }
@@ -1483,7 +1484,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         fut.get();
 
-        awaitFileClose(igfs, FILE);
+        awaitFileClose(igfs.asSecondary(), FILE);
 
         if (err.get() != null)
             throw err.get();
@@ -2137,7 +2138,19 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param files Files.
      * @throws Exception If failed.
      */
-    public static void create(Igfs igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files)
+    public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception {
+        create(igfs.asSecondary(), dirs, files);
+    }
+
+    /**
+     * Create the given directories and files in the given IGFS.
+     *
+     * @param igfs IGFS.
+     * @param dirs Directories.
+     * @param files Files.
+     * @throws Exception If failed.
+     */
+    public static void create(IgfsSecondaryFileSystem igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files)
         throws Exception {
         if (dirs != null) {
             for (IgfsPath dir : dirs)
@@ -2162,7 +2175,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Data chunks.
      * @throws IOException In case of IO exception.
      */
-    protected static void createFile(Igfs igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks)
+    protected static void createFile(IgfsSecondaryFileSystem igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks)
         throws IOException {
         OutputStream os = null;
 
@@ -2188,7 +2201,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Data chunks.
      * @throws Exception If failed.
      */
-    protected void createFile(IgniteFs igfs, IgfsPath file, boolean overwrite, long blockSize,
+    protected void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize,
         @Nullable byte[]... chunks) throws Exception {
         IgfsOutputStream os = null;
 
@@ -2200,7 +2213,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         finally {
             U.closeQuiet(os);
 
-            awaitFileClose(igfs, file);
+            awaitFileClose(igfs.asSecondary(), file);
         }
     }
 
@@ -2212,7 +2225,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Data chunks.
      * @throws Exception If failed.
      */
-    protected void appendFile(IgniteFs igfs, IgfsPath file, @Nullable byte[]... chunks)
+    protected void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
         throws Exception {
         IgfsOutputStream os = null;
 
@@ -2224,7 +2237,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         finally {
             U.closeQuiet(os);
 
-            awaitFileClose(igfs, file);
+            awaitFileClose(igfs.asSecondary(), file);
         }
     }
 
@@ -2248,7 +2261,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param igfs IGFS.
      * @param file File.
      */
-    public static void awaitFileClose(Igfs igfs, IgfsPath file) {
+    public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) {
         try {
             igfs.update(file, Collections.singletonMap("prop", "val"));
         }
@@ -2418,7 +2431,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param igfsSecondary Second IGFS.
      * @throws Exception If failed.
      */
-    protected void clear(IgniteFs igfs, IgniteFs igfsSecondary) throws Exception {
+    protected void clear(IgniteFileSystem igfs, IgniteFileSystem igfsSecondary) throws Exception {
         clear(igfs);
 
         if (dual)
@@ -2431,7 +2444,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param igfs IGFS.
      * @throws Exception If failed.
      */
-    public static void clear(IgniteFs igfs) throws Exception {
+    public static void clear(IgniteFileSystem igfs) throws Exception {
         Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap");
 
         workerMapFld.setAccessible(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
new file mode 100644
index 0000000..fd590f5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCachePerBlockLruEvictionPolicySelfTest.java
@@ -0,0 +1,485 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.igfs.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.lang.*;
+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 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.*;
+
+/**
+ * Tests for IGFS per-block LR eviction policy.
+ */
+@SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
+public class IgfsCachePerBlockLruEvictionPolicySelfTest extends IgfsCommonAbstractTest {
+    /** Primary IGFS name. */
+    private static final String IGFS_PRIMARY = "igfs-primary";
+
+    /** Primary IGFS name. */
+    private static final String IGFS_SECONDARY = "igfs-secondary";
+
+    /** Secondary file system REST endpoint configuration map. */
+    private static final Map<String, String> SECONDARY_REST_CFG = new HashMap<String, String>() {{
+        put("type", "tcp");
+        put("port", "11500");
+    }};
+
+    /** File working in PRIMARY mode. */
+    public static final IgfsPath FILE = new IgfsPath("/file");
+
+    /** File working in DUAL mode. */
+    public static final IgfsPath FILE_RMT = new IgfsPath("/fileRemote");
+
+    /** Primary IGFS instances. */
+    private static IgfsImpl igfsPrimary;
+
+    /** Secondary IGFS instance. */
+    private static IgfsImpl secondaryFs;
+
+    /** Primary file system data cache. */
+    private static GridCacheAdapter<IgfsBlockKey, byte[]> dataCache;
+
+    /** Eviction policy */
+    private static CacheIgfsPerBlockLruEvictionPolicy evictPlc;
+
+    /**
+     * Start a grid with the primary file system.
+     *
+     * @throws Exception If failed.
+     */
+    private void startPrimary() throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(IGFS_PRIMARY);
+        igfsCfg.setBlockSize(512);
+        igfsCfg.setDefaultMode(PRIMARY);
+        igfsCfg.setPrefetchBlocks(1);
+        igfsCfg.setSequentialReadsBeforePrefetch(Integer.MAX_VALUE);
+        igfsCfg.setSecondaryFileSystem(secondaryFs.asSecondary());
+
+        Map<String, IgfsMode> pathModes = new HashMap<>();
+
+        pathModes.put(FILE_RMT.toString(), DUAL_SYNC);
+
+        igfsCfg.setPathModes(pathModes);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        evictPlc = new CacheIgfsPerBlockLruEvictionPolicy();
+
+        dataCacheCfg.setEvictionPolicy(evictPlc);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setQueryIndexEnabled(false);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setQueryIndexEnabled(false);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName("grid-primary");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        Ignite g = G.start(cfg);
+
+        igfsPrimary = (IgfsImpl)g.fileSystem(IGFS_PRIMARY);
+
+        dataCache = igfsPrimary.context().kernalContext().cache().internalCache(
+            igfsPrimary.context().configuration().getDataCacheName());
+    }
+
+    /**
+     * Start a grid with the secondary file system.
+     *
+     * @throws Exception If failed.
+     */
+    private void startSecondary() throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(IGFS_SECONDARY);
+        igfsCfg.setBlockSize(512);
+        igfsCfg.setDefaultMode(PRIMARY);
+        igfsCfg.setIpcEndpointConfiguration(SECONDARY_REST_CFG);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setQueryIndexEnabled(false);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        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(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        Ignite g = G.start(cfg);
+
+        secondaryFs = (IgfsImpl)g.fileSystem(IGFS_SECONDARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        try {
+            // Cleanup.
+            igfsPrimary.format();
+
+            while (!dataCache.isEmpty())
+                U.sleep(100);
+
+            checkEvictionPolicy(0, 0);
+        }
+        finally {
+            stopAllGrids(false);
+        }
+    }
+
+    /**
+     * Startup primary and secondary file systems.
+     *
+     * @throws Exception If failed.
+     */
+    private void start() throws Exception {
+        startSecondary();
+        startPrimary();
+
+        evictPlc.setMaxBlocks(0);
+        evictPlc.setMaxSize(0);
+        evictPlc.setExcludePaths(null);
+    }
+
+    /**
+     * Test how evictions are handled for a file working in PRIMARY mode.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFilePrimary() throws Exception {
+        start();
+
+        // Create file in primary mode. It must not be propagated to eviction policy.
+        igfsPrimary.create(FILE, true).close();
+
+        checkEvictionPolicy(0, 0);
+
+        int blockSize = igfsPrimary.info(FILE).blockSize();
+
+        append(FILE, blockSize);
+
+        checkEvictionPolicy(0, 0);
+
+        read(FILE, 0, blockSize);
+
+        checkEvictionPolicy(0, 0);
+    }
+
+    /**
+     * Test how evictions are handled for a file working in PRIMARY mode.
+     *
+     * @throws Exception If failed.
+     */
+    public void testFileDual() throws Exception {
+        start();
+
+        igfsPrimary.create(FILE_RMT, true).close();
+
+        checkEvictionPolicy(0, 0);
+
+        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
+
+        // File write.
+        append(FILE_RMT, blockSize);
+
+        checkEvictionPolicy(1, blockSize);
+
+        // One more write.
+        append(FILE_RMT, blockSize);
+
+        checkEvictionPolicy(2, blockSize * 2);
+
+        // Read.
+        read(FILE_RMT, 0, blockSize);
+
+        checkEvictionPolicy(2, blockSize * 2);
+    }
+
+    /**
+     * Ensure that a DUAL mode file is not propagated to eviction policy
+     *
+     * @throws Exception If failed.
+     */
+    public void testFileDualExclusion() throws Exception {
+        start();
+
+        evictPlc.setExcludePaths(Collections.singleton(FILE_RMT.toString()));
+
+        // Create file in primary mode. It must not be propagated to eviction policy.
+        igfsPrimary.create(FILE_RMT, true).close();
+
+        checkEvictionPolicy(0, 0);
+
+        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
+
+        append(FILE_RMT, blockSize);
+
+        checkEvictionPolicy(0, 0);
+
+        read(FILE_RMT, 0, blockSize);
+
+        checkEvictionPolicy(0, 0);
+    }
+
+    /**
+     * Ensure that exception is thrown in case we are trying to rename file with one exclude setting to the file with
+     * another.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRenameDifferentExcludeSettings() throws Exception {
+        start();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                igfsPrimary.rename(FILE, FILE_RMT);
+
+                return null;
+            }
+        }, IgfsInvalidPathException.class, "Cannot move file to a path with different eviction exclude setting " +
+            "(need to copy and remove)");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                igfsPrimary.rename(FILE_RMT, FILE);
+
+                return null;
+            }
+        }, IgfsInvalidPathException.class, "Cannot move file to a path with different eviction exclude setting " +
+            "(need to copy and remove)");
+    }
+
+    /**
+     * Test eviction caused by too much blocks.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBlockCountEviction() throws Exception {
+        start();
+
+        int blockCnt = 3;
+
+        evictPlc.setMaxBlocks(blockCnt);
+
+        igfsPrimary.create(FILE_RMT, true).close();
+
+        checkEvictionPolicy(0, 0);
+
+        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
+
+        // Write blocks up to the limit.
+        append(FILE_RMT, blockSize * blockCnt);
+
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Write one more block what should cause eviction.
+        append(FILE_RMT, blockSize);
+
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Read the first block.
+        read(FILE_RMT, 0, blockSize);
+
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+        checkMetrics(1, 1);
+    }
+
+    /**
+     * Test eviction caused by too big data size.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDataSizeEviction() throws Exception {
+        start();
+
+        igfsPrimary.create(FILE_RMT, true).close();
+
+        int blockCnt = 3;
+        int blockSize = igfsPrimary.info(FILE_RMT).blockSize();
+
+        evictPlc.setMaxSize(blockSize * blockCnt);
+
+        // Write blocks up to the limit.
+        append(FILE_RMT, blockSize * blockCnt);
+
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Reset metrics.
+        igfsPrimary.resetMetrics();
+
+        // Read the first block what should cause reordering.
+        read(FILE_RMT, 0, blockSize);
+
+        checkMetrics(1, 0);
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Write one more block what should cause eviction of the block 2.
+        append(FILE_RMT, blockSize);
+
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Read the first block.
+        read(FILE_RMT, 0, blockSize);
+
+        checkMetrics(2, 0);
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+
+        // Read the second block (which was evicted).
+        read(FILE_RMT, blockSize, blockSize);
+
+        checkMetrics(3, 1);
+        checkEvictionPolicy(blockCnt, blockCnt * blockSize);
+    }
+
+    /**
+     * Read some data from the given file with the given offset.
+     *
+     * @param path File path.
+     * @param off Offset.
+     * @param len Length.
+     * @throws Exception If failed.
+     */
+    private void read(IgfsPath path, int off, int len) throws Exception {
+        IgfsInputStream is = igfsPrimary.open(path);
+
+        is.readFully(off, new byte[len]);
+
+        is.close();
+    }
+
+    /**
+     * Append some data to the given file.
+     *
+     * @param path File path.
+     * @param len Data length.
+     * @throws Exception If failed.
+     */
+    private void append(IgfsPath path, int len) throws Exception {
+        IgfsOutputStream os = igfsPrimary.append(path, false);
+
+        os.write(new byte[len]);
+
+        os.close();
+    }
+
+    /**
+     * Check metrics counters.
+     *
+     * @param blocksRead Expected blocks read.
+     * @param blocksReadRmt Expected blocks read remote.
+     * @throws Exception If failed.
+     */
+    public void checkMetrics(final long blocksRead, final long blocksReadRmt) throws Exception {
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                IgfsMetrics metrics = igfsPrimary.metrics();
+
+                return metrics.blocksReadTotal() == blocksRead && metrics.blocksReadRemote() == blocksReadRmt;
+            }
+        }, 5000) : "Unexpected metrics [expectedBlocksReadTotal=" + blocksRead + ", actualBlocksReadTotal=" +
+            igfsPrimary.metrics().blocksReadTotal() + ", expectedBlocksReadRemote=" + blocksReadRmt +
+            ", actualBlocksReadRemote=" + igfsPrimary.metrics().blocksReadRemote() + ']';
+    }
+
+    /**
+     * Check eviction policy state.
+     *
+     * @param curBlocks Current blocks.
+     * @param curBytes Current bytes.
+     */
+    private void checkEvictionPolicy(final int curBlocks, final long curBytes) throws IgniteInterruptedCheckedException {
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return evictPlc.getCurrentBlocks() == curBlocks && evictPlc.getCurrentSize() == curBytes;
+            }
+        }, 5000) : "Unexpected counts [expectedBlocks=" + curBlocks + ", actualBlocks=" + evictPlc.getCurrentBlocks() +
+            ", expectedBytes=" + curBytes + ", currentBytes=" + curBytes + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
index bde395b..255e319 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCacheSelfTest.java
@@ -57,13 +57,13 @@ public class IgfsCacheSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setName("igfs");
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
index 2e48648..297cc7b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
-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.*;
@@ -96,7 +95,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
@@ -104,7 +103,7 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setName("igfs");
         igfsCfg.setBlockSize(BLOCK_SIZE);
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 327d7fa..f44a988 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -28,8 +28,8 @@ import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.IgniteFs.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
 
 /**
  * Tests for IGFS working in mode when remote file system exists: DUAL_SYNC, DUAL_ASYNC.
@@ -1120,7 +1120,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         create(igfsSecondary, paths(DIR, SUBDIR), null);
         create(igfs, null, null);
 
-        createFile(igfsSecondary, FILE, true, chunk);
+        createFile(igfsSecondary.asSecondary(), FILE, true, chunk);
 
         checkFileContent(igfs, FILE, chunk);
     }
@@ -1148,7 +1148,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         out.close();
 
-        awaitFileClose(igfsSecondary, FILE);
+        awaitFileClose(igfsSecondary.asSecondary(), FILE);
 
         // Read the first block.
         int totalRead = 0;
@@ -1227,7 +1227,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         out.close();
 
-        awaitFileClose(igfsSecondary, FILE);
+        awaitFileClose(igfsSecondary.asSecondary(), FILE);
 
         // Read the first two blocks.
         int totalRead = 0;
@@ -1292,7 +1292,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
         igfsSecondary.update(SUBDIR, props);
 
-        createFile(igfs, FILE, true, chunk);
+        createFile(igfs.asSecondary(), FILE, true, chunk);
 
         // Ensure that directory structure was created.
         checkExist(igfs, igfsSecondary, SUBDIR);
@@ -1317,7 +1317,7 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
         igfsSecondary.update(DIR, propsDir);
         igfsSecondary.update(SUBDIR, propsSubDir);
 
-        createFile(igfs, FILE, true, chunk);
+        createFile(igfs.asSecondary(), FILE, true, chunk);
 
         checkExist(igfs, igfsSecondary, SUBDIR);
         checkFile(igfs, igfsSecondary, FILE, chunk);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 1a52b7c..11d6cc5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -75,13 +75,13 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setName("igfs");
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
index be5afe7..0af1dea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
@@ -56,10 +56,10 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** Primary IGFS instances. */
-    private static IgniteFs[] igfsPrimary;
+    private static IgniteFileSystem[] igfsPrimary;
 
     /** Secondary IGFS instance. */
-    private static IgniteFs igfsSecondary;
+    private static IgfsImpl igfsSecondary;
 
     /** Primary file system block size. */
     public static final int PRIMARY_BLOCK_SIZE = 512;
@@ -84,7 +84,7 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     private void startPrimary() throws Exception {
-        igfsPrimary = new IgniteFs[NODES_CNT];
+        igfsPrimary = new IgniteFileSystem[NODES_CNT];
 
         for (int i = 0; i < NODES_CNT; i++) {
             Ignite g = G.start(primaryConfiguration(i));
@@ -101,14 +101,14 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     private IgniteConfiguration primaryConfiguration(int idx) throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
         igfsCfg.setName(IGFS_PRIMARY);
         igfsCfg.setBlockSize(PRIMARY_BLOCK_SIZE);
         igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setSecondaryFileSystem(igfsSecondary);
+        igfsCfg.setSecondaryFileSystem(igfsSecondary.asSecondary());
 
         Map<String, IgfsMode> pathModes = new HashMap<>();
 
@@ -146,7 +146,7 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
 
@@ -159,7 +159,7 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     private void startSecondary() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
@@ -198,18 +198,18 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
 
         Ignite g = G.start(cfg);
 
-        igfsSecondary = g.fileSystem(IGFS_SECONDARY);
+        igfsSecondary = (IgfsImpl)g.fileSystem(IGFS_SECONDARY);
     }
 
     /** @throws Exception If failed. */
     public void testMetrics() throws Exception {
-        IgniteFs fs = igfsPrimary[0];
+        IgniteFileSystem fs = igfsPrimary[0];
 
         assertNotNull(fs);
 
@@ -349,7 +349,7 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testMultipleClose() throws Exception {
-        IgniteFs fs = igfsPrimary[0];
+        IgniteFileSystem fs = igfsPrimary[0];
 
         IgfsOutputStream out = fs.create(new IgfsPath("/file"), false);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
index 2b501f7..4a58285 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModesSelfTest.java
@@ -82,7 +82,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
     private void startUp() throws Exception {
         startUpSecondary();
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("partitioned");
         igfsCfg.setMetaCacheName("replicated");
@@ -97,7 +97,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setPathModes(pathModes);
 
         if (setSecondaryFs)
-            igfsCfg.setSecondaryFileSystem(igfsSecondary);
+            igfsCfg.setSecondaryFileSystem(igfsSecondary.asSecondary());
 
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 
@@ -128,7 +128,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
         cfg.setConnectorConfiguration(null);
@@ -144,7 +144,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     private void startUpSecondary() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("partitioned");
         igfsCfg.setMetaCacheName("replicated");
@@ -185,7 +185,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setLocalHost("127.0.0.1");
         cfg.setConnectorConfiguration(null);
@@ -378,7 +378,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
         }
 
         assertTrue(errMsg.startsWith(
-            "Grid configuration parameter invalid: secondaryFileSystem cannot be null when mode is SECONDARY"));
+            "Grid configuration parameter invalid: secondaryFileSystem cannot be null when mode is not PRIMARY"));
     }
 
     /**
@@ -444,7 +444,7 @@ public class IgfsModesSelfTest extends IgfsCommonAbstractTest {
         }
 
         assertTrue(errMsg.startsWith(
-            "Grid configuration parameter invalid: secondaryFileSystem cannot be null when mode is SECONDARY"));
+            "Grid configuration parameter invalid: secondaryFileSystem cannot be null when mode is not PRIMARY"));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index ead7511..0e03aa8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -59,7 +59,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
     protected final SecureRandom rnd = new SecureRandom();
 
     /** File system. */
-    protected IgniteFs igfs;
+    protected IgniteFileSystem igfs;
 
     /** Meta cache. */
     private GridCache<Object, Object> metaCache;
@@ -73,7 +73,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
         igfs = grid.fileSystem(igfsName());
 
-        IgfsConfiguration[] cfgs = grid.configuration().getIgfsConfiguration();
+        FileSystemConfiguration[] cfgs = grid.configuration().getFileSystemConfiguration();
 
         assert cfgs.length == 1;
 
@@ -111,13 +111,13 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setName("igfs");
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }
@@ -157,7 +157,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
     /** @throws Exception If failed. */
     public void testigfsEnabled() throws Exception {
-        IgniteFs igfs = grid(0).fileSystem(igfsName());
+        IgniteFileSystem igfs = grid(0).fileSystem(igfsName());
 
         assertNotNull(igfs);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
index 6f444c0..dcd0413 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
@@ -37,7 +37,7 @@ import static org.apache.ignite.igfs.IgfsMode.*;
  * Tests for node validation logic in {@link IgfsProcessor}.
  * <p>
  * Tests starting with "testLocal" are checking
- * {@link IgfsProcessor#validateLocalIgfsConfigurations(org.apache.ignite.configuration.IgfsConfiguration[])}.
+ * {@link IgfsProcessor#validateLocalIgfsConfigurations(org.apache.ignite.configuration.FileSystemConfiguration[])}.
  * <p>
  * Tests starting with "testRemote" are checking {@link IgfsProcessor#checkIgfsOnRemoteNode(org.apache.ignite.cluster.ClusterNode)}.
  */
@@ -61,10 +61,10 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     private static final String metaCache2Name = "metaCache2";
 
     /** First IGFS config in grid #1. */
-    private IgfsConfiguration g1IgfsCfg1 = new IgfsConfiguration();
+    private FileSystemConfiguration g1IgfsCfg1 = new FileSystemConfiguration();
 
     /** Second IGFS config in grid#1. */
-    private IgfsConfiguration g1IgfsCfg2 = new IgfsConfiguration();
+    private FileSystemConfiguration g1IgfsCfg2 = new FileSystemConfiguration();
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
@@ -89,7 +89,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g1IgfsCfg2.setDataCacheName(dataCache2Name);
         g1IgfsCfg2.setMetaCacheName(metaCache2Name);
 
-        cfg.setIgfsConfiguration(g1IgfsCfg1, g1IgfsCfg2);
+        cfg.setFileSystemConfiguration(g1IgfsCfg1, g1IgfsCfg2);
 
         cfg.setLocalHost("127.0.0.1");
 
@@ -266,7 +266,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
 
         g1IgfsCfg2.setDefaultMode(PROXY);
 
-        checkGridStartFails(g1Cfg, "secondaryFileSystem cannot be null when mode is SECONDARY", true);
+        checkGridStartFails(g1Cfg, "secondaryFileSystem cannot be null when mode is not PRIMARY", true);
     }
 
     /**
@@ -278,11 +278,11 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
 
         g2IgfsCfg1.setBlockSize(g2IgfsCfg1.getBlockSize() + 100);
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g1IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g1IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -310,8 +310,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfMetaCacheNameDiffers() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g2IgfsCfg1.setMetaCacheName("g2MetaCache1");
         g2IgfsCfg2.setMetaCacheName("g2MetaCache2");
@@ -320,7 +320,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches("g2MetaCache1", "g2MetaCache2"),
              CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -333,8 +333,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfMetaCacheNameEquals() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g2IgfsCfg1.setName("g2IgfsCfg1");
         g2IgfsCfg2.setName("g2IgfsCfg2");
@@ -346,7 +346,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024, "g2DataCache1", "g2DataCache2"), metaCaches(),
              CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -359,8 +359,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfDataCacheNameDiffers() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g2IgfsCfg1.setDataCacheName("g2DataCache1");
         g2IgfsCfg2.setDataCacheName("g2DataCache2");
@@ -369,7 +369,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024, "g2DataCache1", "g2DataCache2"), metaCaches(),
              CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -382,8 +382,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfDataCacheNameEquals() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g2IgfsCfg1.setName("g2IgfsCfg1");
         g2IgfsCfg2.setName("g2IgfsCfg2");
@@ -395,7 +395,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches("g2MetaCache1", "g2MetaCache2"),
              CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -408,8 +408,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfDefaultModeDiffers() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g1IgfsCfg1.setDefaultMode(DUAL_ASYNC);
         g1IgfsCfg2.setDefaultMode(DUAL_ASYNC);
@@ -420,7 +420,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 
@@ -433,8 +433,8 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     public void testRemoteIfPathModeDiffers() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 
-        IgfsConfiguration g2IgfsCfg1 = new IgfsConfiguration(g1IgfsCfg1);
-        IgfsConfiguration g2IgfsCfg2 = new IgfsConfiguration(g1IgfsCfg2);
+        FileSystemConfiguration g2IgfsCfg1 = new FileSystemConfiguration(g1IgfsCfg1);
+        FileSystemConfiguration g2IgfsCfg2 = new FileSystemConfiguration(g1IgfsCfg2);
 
         g2IgfsCfg1.setPathModes(Collections.singletonMap("/somePath", DUAL_SYNC));
         g2IgfsCfg2.setPathModes(Collections.singletonMap("/somePath", DUAL_SYNC));
@@ -442,7 +442,7 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
         g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
         g2Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
 
-        g2Cfg.setIgfsConfiguration(g2IgfsCfg1, g2IgfsCfg2);
+        g2Cfg.setFileSystemConfiguration(g2IgfsCfg1, g2IgfsCfg2);
 
         G.start(g1Cfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
index ee8e7bd..694d5c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
@@ -36,7 +36,7 @@ import java.util.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
 
 /**
  * Base test class for {@link IgfsServer} checking IPC endpoint registrations.
@@ -58,7 +58,7 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
     public void testLoopbackEndpointsRegistration() throws Exception {
         IgniteConfiguration cfg = gridConfiguration();
 
-        cfg.setIgfsConfiguration(
+        cfg.setFileSystemConfiguration(
             igfsConfiguration("tcp", DFLT_IPC_PORT, null)
         );
 
@@ -77,7 +77,7 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
     public void testLoopbackEndpointsCustomHostRegistration() throws Exception {
         IgniteConfiguration cfg = gridConfiguration();
 
-        cfg.setIgfsConfiguration(
+        cfg.setFileSystemConfiguration(
             igfsConfiguration("tcp", DFLT_IPC_PORT, "127.0.0.1"),
             igfsConfiguration("tcp", DFLT_IPC_PORT + 1, U.getLocalHost().getHostName()));
 
@@ -154,7 +154,7 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
      * @param endPntHost End point host.
      * @return test-purposed IgfsConfiguration.
      */
-    protected IgfsConfiguration igfsConfiguration(@Nullable String endPntType, @Nullable Integer endPntPort,
+    protected FileSystemConfiguration igfsConfiguration(@Nullable String endPntType, @Nullable Integer endPntPort,
         @Nullable String endPntHost) throws IgniteCheckedException {
         HashMap<String, String> endPntCfg = null;
 
@@ -170,7 +170,7 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
                 endPntCfg.put("host", endPntHost);
         }
 
-        IgfsConfiguration igfsConfiguration = new IgfsConfiguration();
+        FileSystemConfiguration igfsConfiguration = new FileSystemConfiguration();
 
         igfsConfiguration.setDataCacheName("partitioned");
         igfsConfiguration.setMetaCacheName("replicated");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
index d7319f4..57f10d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.igfs;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
 
 /**
  * Tests for {@link IgfsServer} that checks all IPC endpoint registration types
@@ -34,7 +34,7 @@ public class IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest
     public void testLoopbackAndShmemEndpointsRegistration() throws Exception {
         IgniteConfiguration cfg = gridConfiguration();
 
-        cfg.setIgfsConfiguration(
+        cfg.setFileSystemConfiguration(
             igfsConfiguration(null, null, null), // Check null IPC endpoint config won't bring any hassles.
             igfsConfiguration("tcp", DFLT_IPC_PORT + 1, null),
             igfsConfiguration("shmem", DFLT_IPC_PORT + 2, null));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest.java
index 259f31a..4f18aff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest.java
@@ -40,7 +40,7 @@ public class IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest
             @Override public Object call() throws Exception {
                 IgniteConfiguration cfg = gridConfiguration();
 
-                cfg.setIgfsConfiguration(igfsConfiguration("shmem", IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT,
+                cfg.setFileSystemConfiguration(igfsConfiguration("shmem", IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT,
                     null));
 
                 return G.start(cfg);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
index b212f02..c62e759 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
@@ -111,7 +111,7 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
@@ -154,7 +154,7 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(metaCfg, dataCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
index 054665d..25816c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
@@ -74,7 +74,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
     public static final int ASSERT_RETRY_INTERVAL = 100;
 
     /** File system to test. */
-    private IgniteFs fs;
+    private IgniteFileSystem fs;
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
@@ -110,7 +110,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
         igfsCfg.setDataCacheName(DATA_CACHE_NAME);
@@ -118,7 +118,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setBlockSize(CFG_BLOCK_SIZE);
         igfsCfg.setFragmentizerEnabled(true);
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }
@@ -229,12 +229,12 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
         IgfsPath path = new IgfsPath("/file");
 
         try {
-            IgniteFs fs0 = grid(0).fileSystem("igfs");
-            IgniteFs fs1 = grid(1).fileSystem("igfs");
-            IgniteFs fs2 = grid(2).fileSystem("igfs");
+            IgniteFileSystem fs0 = grid(0).fileSystem("igfs");
+            IgniteFileSystem fs1 = grid(1).fileSystem("igfs");
+            IgniteFileSystem fs2 = grid(2).fileSystem("igfs");
 
             try (IgfsOutputStream out = fs0.create(path, 128, false, 1, CFG_GRP_SIZE,
-                F.asMap(IgniteFs.PROP_PREFER_LOCAL_WRITES, "true"))) {
+                F.asMap(IgfsEx.PROP_PREFER_LOCAL_WRITES, "true"))) {
                 // 1.5 blocks
                 byte[] data = new byte[CFG_BLOCK_SIZE * 3 / 2];
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
index 5b7a636..d1778df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsTaskSelfTest.java
@@ -68,7 +68,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
     private static final int REPEAT_CNT = 10;
 
     /** IGFS. */
-    private static IgniteFs igfs;
+    private static IgniteFileSystem igfs;
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
@@ -97,7 +97,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
      * @return Grid configuration
      */
     private IgniteConfiguration config(int idx) {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
@@ -133,7 +133,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setGridName("node-" + idx);
 
@@ -172,7 +172,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
 
         assertFalse(igfs.isAsync());
 
-        IgniteFs igfsAsync = igfs.withAsync();
+        IgniteFileSystem igfsAsync = igfs.withAsync();
 
         assertTrue(igfsAsync.isAsync());
 
@@ -269,7 +269,7 @@ public class IgfsTaskSelfTest extends IgfsCommonAbstractTest {
         private ComputeJobContext ctx;
 
         /** {@inheritDoc} */
-        @Override public Object execute(IgniteFs igfs, IgfsFileRange range, IgfsInputStream in)
+        @Override public Object execute(IgniteFileSystem igfs, IgfsFileRange range, IgfsInputStream in)
             throws IOException {
             assert ignite != null;
             assert ses != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/split/IgfsAbstractRecordResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/split/IgfsAbstractRecordResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/split/IgfsAbstractRecordResolverSelfTest.java
index 05de61a..487d391 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/split/IgfsAbstractRecordResolverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/split/IgfsAbstractRecordResolverSelfTest.java
@@ -46,11 +46,11 @@ public class IgfsAbstractRecordResolverSelfTest extends GridCommonAbstractTest {
     private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** IGFS. */
-    protected static IgniteFs igfs;
+    protected static IgniteFileSystem igfs;
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
@@ -87,7 +87,7 @@ public class IgfsAbstractRecordResolverSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         Ignite g = G.start(cfg);
 


[43/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/IgfsReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsReader.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsReader.java
deleted file mode 100644
index fff8ca3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsReader.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.igfs;
-
-import java.io.*;
-
-/**
- * The simplest data input interface to read from secondary file system in dual modes.
- */
-public interface IgfsReader extends Closeable {
-    /**
-     * Read up to the specified number of bytes, from a given position within a file, and return the number of bytes
-     * read.
-     *
-     * @param pos Position in the input stream to seek.
-     * @param buf Buffer into which data is read.
-     * @param off Offset in the buffer from which stream data should be written.
-     * @param len The number of bytes to read.
-     * @return Total number of bytes read into the buffer, or -1 if there is no more data (EOF).
-     * @throws IOException In case of any exception.
-     */
-    public int read(long pos, byte[] buf, int off, int len) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsInputStreamJobAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsInputStreamJobAdapter.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsInputStreamJobAdapter.java
index bf2636f..d7fd866 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsInputStreamJobAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsInputStreamJobAdapter.java
@@ -32,7 +32,7 @@ import java.io.*;
  */
 public abstract class IgfsInputStreamJobAdapter extends IgfsJobAdapter {
     /** {@inheritDoc} */
-    @Override public final Object execute(IgniteFs igfs, IgfsFileRange range, IgfsInputStream in)
+    @Override public final Object execute(IgniteFileSystem igfs, IgfsFileRange range, IgfsInputStream in)
         throws IgniteException, IOException {
         in.seek(range.start());
 
@@ -48,5 +48,5 @@ public abstract class IgfsInputStreamJobAdapter extends IgfsJobAdapter {
      * @throws IgniteException If execution failed.
      * @throws IOException If IO exception encountered while working with stream.
      */
-    public abstract Object execute(IgniteFs igfs, IgfsRangeInputStream in) throws IgniteException, IOException;
+    public abstract Object execute(IgniteFileSystem igfs, IgfsRangeInputStream in) throws IgniteException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsJob.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsJob.java
index d12fc79..0fdc0f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsJob.java
@@ -26,7 +26,7 @@ import java.io.*;
  * Defines executable unit for {@link IgfsTask}. Before this job is executed, it is assigned one of the
  * ranges provided by the {@link IgfsRecordResolver} passed to one of the {@code IgniteFs.execute(...)} methods.
  * <p>
- * {@link #execute(org.apache.ignite.IgniteFs, IgfsFileRange, org.apache.ignite.igfs.IgfsInputStream)} method is given {@link IgfsFileRange} this
+ * {@link #execute(org.apache.ignite.IgniteFileSystem, IgfsFileRange, org.apache.ignite.igfs.IgfsInputStream)} method is given {@link IgfsFileRange} this
  * job is expected to operate on, and already opened {@link org.apache.ignite.igfs.IgfsInputStream} for the file this range belongs to.
  * <p>
  * Note that provided input stream has position already adjusted to range start. However, it will not
@@ -52,7 +52,7 @@ public interface IgfsJob {
      * @throws IgniteException If execution failed.
      * @throws IOException If file system operation resulted in IO exception.
      */
-    public Object execute(IgniteFs igfs, IgfsFileRange range, IgfsInputStream in) throws IgniteException,
+    public Object execute(IgniteFileSystem igfs, IgfsFileRange range, IgfsInputStream in) throws IgniteException,
         IOException;
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsRecordResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsRecordResolver.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsRecordResolver.java
index 52afeda..cc1d73f 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsRecordResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsRecordResolver.java
@@ -51,6 +51,6 @@ public interface IgfsRecordResolver extends Serializable {
      * @throws IgniteException If resolve failed.
      * @throws IOException If resolve failed.
      */
-    @Nullable public IgfsFileRange resolveRecords(IgniteFs fs, IgfsInputStream stream,
+    @Nullable public IgfsFileRange resolveRecords(IgniteFileSystem fs, IgfsInputStream stream,
         IgfsFileRange suggestedRecord) throws IgniteException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTask.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTask.java
index e42b012..9936140 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTask.java
@@ -37,7 +37,7 @@ import java.util.*;
  * <p>
  * Each file participating in IGFS task is split into {@link IgfsFileRange}s first. Normally range is a number of
  * consequent bytes located on a single node (see {@code IgfssGroupDataBlocksKeyMapper}). In case maximum range size
- * is provided (either through {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()} or {@code IgniteFs.execute()}
+ * is provided (either through {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()} or {@code IgniteFs.execute()}
  * argument), then ranges could be further divided into smaller chunks.
  * <p>
  * Once file is split into ranges, each range is passed to {@code IgfsTask.createJob()} method in order to create a
@@ -88,7 +88,7 @@ public abstract class IgfsTask<T, R> extends ComputeTaskAdapter<IgfsTaskArgs<T>,
         assert ignite != null;
         assert args != null;
 
-        IgniteFs fs = ignite.fileSystem(args.igfsName());
+        IgniteFileSystem fs = ignite.fileSystem(args.igfsName());
         IgfsProcessorAdapter igfsProc = ((IgniteKernal) ignite).context().igfs();
 
         Map<ComputeJob, ClusterNode> splitMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTaskArgs.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTaskArgs.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTaskArgs.java
index 7db26ec..5ef5352 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTaskArgs.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/IgfsTaskArgs.java
@@ -33,7 +33,7 @@ import java.util.*;
  *     <li>{@link IgfsRecordResolver} for that task</li>
  *     <li>Flag indicating whether to skip non-existent file paths or throw an exception</li>
  *     <li>User-defined task argument</li>
- *     <li>Maximum file range length for that task (see {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()})</li>
+ *     <li>Maximum file range length for that task (see {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()})</li>
  * </ul>
  */
 public interface IgfsTaskArgs<T> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsByteDelimiterRecordResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsByteDelimiterRecordResolver.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsByteDelimiterRecordResolver.java
index c1c15a5..c9ed821 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsByteDelimiterRecordResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsByteDelimiterRecordResolver.java
@@ -20,7 +20,6 @@ package org.apache.ignite.igfs.mapreduce.records;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.mapreduce.*;
-import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -78,7 +77,7 @@ public class IgfsByteDelimiterRecordResolver implements IgfsRecordResolver, Exte
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFileRange resolveRecords(IgniteFs fs, IgfsInputStream stream,
+    @Override public IgfsFileRange resolveRecords(IgniteFileSystem fs, IgfsInputStream stream,
         IgfsFileRange suggestedRecord) throws IgniteException, IOException {
         long suggestedStart = suggestedRecord.start();
         long suggestedEnd = suggestedStart + suggestedRecord.length();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsFixedLengthRecordResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsFixedLengthRecordResolver.java b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsFixedLengthRecordResolver.java
index fe22627..bcc8f69 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsFixedLengthRecordResolver.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/mapreduce/records/IgfsFixedLengthRecordResolver.java
@@ -52,7 +52,7 @@ public class IgfsFixedLengthRecordResolver implements IgfsRecordResolver, Extern
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFileRange resolveRecords(IgniteFs fs, IgfsInputStream stream,
+    @Override public IgfsFileRange resolveRecords(IgniteFileSystem fs, IgfsInputStream stream,
         IgfsFileRange suggestedRecord)
         throws IgniteException, IOException {
         long suggestedEnd = suggestedRecord.start() + suggestedRecord.length();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
new file mode 100644
index 0000000..089a8e3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.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.igfs.secondary;
+
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Secondary file system interface.
+ */
+public interface IgfsSecondaryFileSystem {
+    /**
+     * Checks if the specified path exists.
+     *
+     * @param path Path to check for existence.
+     * @return {@code True} if such file exists, otherwise - {@code false}.
+     * @throws IgniteException In case of error.
+     */
+    public boolean exists(IgfsPath path);
+
+    /**
+     * Updates file information for the specified path. Existent properties, not listed in the passed collection,
+     * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values
+     * will be removed from the stored properties or ignored if they don't exist in the file info.
+     * <p>
+     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated
+     * to the secondary file system:
+     * <ul>
+     * <li>{@code usrName} - file owner name;</li>
+     * <li>{@code grpName} - file owner group;</li>
+     * <li>{@code permission} - Unix-style string representing file permissions.</li>
+     * </ul>
+     *
+     * @param path File path to set properties for.
+     * @param props Properties to update.
+     * @return File information for specified path or {@code null} if such path does not exist.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteException;
+
+    /**
+     * Renames/moves a file.
+     * <p>
+     * You are free to rename/move data files as you wish, but directories can be only renamed.
+     * You cannot move the directory between different parent directories.
+     * <p>
+     * Examples:
+     * <ul>
+     *     <li>"/work/file.txt" => "/home/project/Presentation Scenario.txt"</li>
+     *     <li>"/work" => "/work-2012.bkp"</li>
+     *     <li>"/work" => "<strike>/backups/work</strike>" - such operation is restricted for directories.</li>
+     * </ul>
+     *
+     * @param src Source file path to rename.
+     * @param dest Destination file path. If destination path is a directory, then source file will be placed
+     *     into destination directory with original name.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If source file doesn't exist.
+     */
+    public void rename(IgfsPath src, IgfsPath dest) throws IgniteException;
+
+    /**
+     * Deletes file.
+     *
+     * @param path File path to delete.
+     * @param recursive Delete non-empty directories recursively.
+     * @return {@code True} in case of success, {@code false} otherwise.
+     * @throws IgniteException In case of error.
+     */
+    public boolean delete(IgfsPath path, boolean recursive) throws IgniteException;
+
+    /**
+     * Creates directories under specified path.
+     *
+     * @param path Path of directories chain to create.
+     * @throws IgniteException In case of error.
+     */
+    public void mkdirs(IgfsPath path) throws IgniteException;
+
+    /**
+     * Creates directories under specified path with the specified properties.
+     *
+     * @param path Path of directories chain to create.
+     * @param props Metadata properties to set on created directories.
+     * @throws IgniteException In case of error.
+     */
+    public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) throws IgniteException;
+
+    /**
+     * Lists file paths under the specified path.
+     *
+     * @param path Path to list files under.
+     * @return List of files under the specified path.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteException;
+
+    /**
+     * Lists files under the specified path.
+     *
+     * @param path Path to list files under.
+     * @return List of files under the specified path.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteException;
+
+    /**
+     * Opens a file for reading.
+     *
+     * @param path File path to read.
+     * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
+     * @return File input stream to read data from.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist.
+     */
+    public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) throws IgniteException;
+
+    /**
+     * Creates a file and opens it for writing.
+     *
+     * @param path File path to create.
+     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
+     * @return File output stream to write data to.
+     * @throws IgniteException In case of error.
+     */
+    public OutputStream create(IgfsPath path, boolean overwrite) throws IgniteException;
+
+    /**
+     * Creates a file and opens it for writing.
+     *
+     * @param path File path to create.
+     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
+     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
+     * @param replication Replication factor.
+     * @param blockSize Block size.
+     * @param props File properties to set.
+     * @return File output stream to write data to.
+     * @throws IgniteException In case of error.
+     */
+    public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, long blockSize,
+       @Nullable Map<String, String> props) throws IgniteException;
+
+    /**
+     * Opens an output stream to an existing file for appending data.
+     *
+     * @param path File path to append.
+     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
+     * @param create Create file if it doesn't exist yet.
+     * @param props File properties to set only in case it file was just created.
+     * @return File output stream to append data to.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist and create flag is {@code false}.
+     */
+    public OutputStream append(IgfsPath path, int bufSize, boolean create, @Nullable Map<String, String> props)
+        throws IgniteException;
+
+    /**
+     * Gets file information for the specified path.
+     *
+     * @param path Path to get information for.
+     * @return File information for specified path or {@code null} if such path does not exist.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsFile info(IgfsPath path) throws IgniteException;
+
+    /**
+     * Gets used space in bytes.
+     *
+     * @return Used space in bytes.
+     * @throws IgniteException In case of error.
+     */
+    public long usedSpaceSize() throws IgniteException;
+
+    /**
+     * Gets the implementation specific properties of file system.
+     *
+     * @return Map of properties.
+     */
+    public Map<String,String> properties();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystemPositionedReadable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystemPositionedReadable.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystemPositionedReadable.java
new file mode 100644
index 0000000..3d36236
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystemPositionedReadable.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.igfs.secondary;
+
+import java.io.*;
+
+/**
+ * The simplest data input interface to read from secondary file system.
+ */
+public interface IgfsSecondaryFileSystemPositionedReadable extends Closeable {
+    /**
+     * Read up to the specified number of bytes, from a given position within a file, and return the number of bytes
+     * read.
+     *
+     * @param pos Position in the input stream to seek.
+     * @param buf Buffer into which data is read.
+     * @param off Offset in the buffer from which stream data should be written.
+     * @param len The number of bytes to read.
+     * @return Total number of bytes read into the buffer, or -1 if there is no more data (EOF).
+     * @throws IOException In case of any exception.
+     */
+    public int read(long pos, byte[] buf, int off, int len) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package.html b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package.html
new file mode 100644
index 0000000..8bd668d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package.html
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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 APIs for IGFS secondary file system.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index cb9ffa1..30ba883 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -282,7 +282,7 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      *
      * @return Hadoop processor.
      */
-    public IgniteHadoopProcessorAdapter hadoop();
+    public HadoopProcessorAdapter hadoop();
 
     /**
      * Gets utility cache pool.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 756c16a..e80df0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -230,7 +230,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
-    private IgniteHadoopProcessorAdapter hadoopProc;
+    private HadoopProcessorAdapter hadoopProc;
 
     /** */
     @GridToStringExclude
@@ -456,8 +456,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             streamProc = (GridStreamProcessor)comp;
         else if (comp instanceof GridContinuousProcessor)
             contProc = (GridContinuousProcessor)comp;
-        else if (comp instanceof IgniteHadoopProcessorAdapter)
-            hadoopProc = (IgniteHadoopProcessorAdapter)comp;
+        else if (comp instanceof HadoopProcessorAdapter)
+            hadoopProc = (HadoopProcessorAdapter)comp;
         else if (comp instanceof GridPortableProcessor)
             portableProc = (GridPortableProcessor)comp;
         else if (comp instanceof IgnitePluginProcessor)
@@ -680,7 +680,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteHadoopProcessorAdapter hadoop() {
+    @Override public HadoopProcessorAdapter hadoop() {
         return hadoopProc;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..0e5c1cf 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
@@ -35,8 +35,8 @@ 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.HadoopNoopProcessor",
+        "org.apache.ignite.internal.processors.hadoop.HadoopProcessor",
         "ignite-hadoop"
     ),
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 3c35a08..3ede8d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -117,14 +117,14 @@ public interface IgniteEx extends Ignite {
      * @param name IGFS name.
      * @return IGFS.
      */
-    @Nullable public IgniteFs igfsx(@Nullable String name);
+    @Nullable public IgniteFileSystem igfsx(@Nullable String name);
 
     /**
      * Get Hadoop facade.
      *
      * @return Hadoop.
      */
-    public GridHadoop hadoop();
+    public Hadoop hadoop();
 
     /** {@inheritDoc} */
     @Override IgniteClusterEx cluster();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index b9abdb5..76c070d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -704,7 +704,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Starts lifecycle aware components.
             U.startLifecycleAware(lifecycleAwares(cfg));
 
-            addHelper(ctx, IGFS_HELPER.create(F.isEmpty(cfg.getIgfsConfiguration())));
+            addHelper(ctx, IGFS_HELPER.create(F.isEmpty(cfg.getFileSystemConfiguration())));
 
             startProcessor(ctx, new IgnitePluginProcessor(ctx, cfg), attrs);
 
@@ -756,7 +756,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(ctx, new GridRestProcessor(ctx), attrs);
             startProcessor(ctx, new GridDataLoaderProcessor(ctx), attrs);
             startProcessor(ctx, new GridStreamProcessor(ctx), attrs);
-            startProcessor(ctx, (GridProcessor) IGFS.create(ctx, F.isEmpty(cfg.getIgfsConfiguration())), attrs);
+            startProcessor(ctx, (GridProcessor) IGFS.create(ctx, F.isEmpty(cfg.getFileSystemConfiguration())), attrs);
             startProcessor(ctx, new GridContinuousProcessor(ctx), attrs);
             startProcessor(ctx, (GridProcessor)(cfg.isPeerClassLoadingEnabled() ?
                 IgniteComponentType.HADOOP.create(ctx, true): // No-op when peer class loading is enabled.
@@ -2349,11 +2349,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFs fileSystem(String name) {
+    @Override public IgniteFileSystem fileSystem(String name) {
         guard();
 
         try{
-            IgniteFs fs = ctx.igfs().igfs(name);
+            IgniteFileSystem fs = ctx.igfs().igfs(name);
 
             if (fs == null)
                 throw new IllegalArgumentException("IGFS is not configured: " + name);
@@ -2366,7 +2366,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFs igfsx(@Nullable String name) {
+    @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
         guard();
 
         try {
@@ -2378,7 +2378,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFs> fileSystems() {
+    @Override public Collection<IgniteFileSystem> fileSystems() {
         guard();
 
         try {
@@ -2390,7 +2390,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public GridHadoop hadoop() {
+    @Override public Hadoop hadoop() {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index cb2efbf..2e8cfc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1606,15 +1606,15 @@ public class IgnitionEx {
             if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
                 myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
 
-            IgfsConfiguration[] igfsCfgs = myCfg.getIgfsConfiguration();
+            FileSystemConfiguration[] igfsCfgs = myCfg.getFileSystemConfiguration();
 
             if (igfsCfgs != null) {
-                IgfsConfiguration[] clone = igfsCfgs.clone();
+                FileSystemConfiguration[] clone = igfsCfgs.clone();
 
                 for (int i = 0; i < igfsCfgs.length; i++)
-                    clone[i] = new IgfsConfiguration(igfsCfgs[i]);
+                    clone[i] = new FileSystemConfiguration(igfsCfgs[i]);
 
-                myCfg.setIgfsConfiguration(clone);
+                myCfg.setFileSystemConfiguration(clone);
             }
 
             StreamerConfiguration[] streamerCfgs = myCfg.getStreamerConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 12ea535..fe88012 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -236,10 +236,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         mxBean = new CacheMetricsMXBeanImpl(this);
 
-        IgfsConfiguration[] igfsCfgs = gridCfg.getIgfsConfiguration();
+        FileSystemConfiguration[] igfsCfgs = gridCfg.getFileSystemConfiguration();
 
         if (igfsCfgs != null) {
-            for (IgfsConfiguration igfsCfg : igfsCfgs) {
+            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
                 if (F.eq(ctx.name(), igfsCfg.getDataCacheName())) {
                     if (!ctx.isNear()) {
                         igfsDataCache = true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 3b2ca47..72c2b49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -550,10 +550,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         maxPreloadOrder = validatePreloadOrder(ctx.config().getCacheConfiguration());
 
         // Internal caches which should not be returned to user.
-        IgfsConfiguration[] igfsCfgs = ctx.grid().configuration().getIgfsConfiguration();
+        FileSystemConfiguration[] igfsCfgs = ctx.grid().configuration().getFileSystemConfiguration();
 
         if (igfsCfgs != null) {
-            for (IgfsConfiguration igfsCfg : igfsCfgs) {
+            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
                 sysCaches.add(igfsCfg.getMetaCacheName());
                 sysCaches.add(igfsCfg.getDataCacheName());
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 1e67907..b07c14e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1594,10 +1594,10 @@ public class GridCacheUtils {
      * @return {@code True} in this is IGFS data or meta cache.
      */
     public static boolean isIgfsCache(IgniteConfiguration cfg, @Nullable String cacheName) {
-        IgfsConfiguration[] igfsCfgs = cfg.getIgfsConfiguration();
+        FileSystemConfiguration[] igfsCfgs = cfg.getFileSystemConfiguration();
 
         if (igfsCfgs != null) {
-            for (IgfsConfiguration igfsCfg : igfsCfgs) {
+            for (FileSystemConfiguration igfsCfg : igfsCfgs) {
                 // IGFS config probably has not been validated yet => possible NPE, so we check for null.
                 if (igfsCfg != null &&
                     (F.eq(cacheName, igfsCfg.getDataCacheName()) || F.eq(cacheName, igfsCfg.getMetaCacheName())))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoop.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoop.java
deleted file mode 100644
index c262d48..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoop.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;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Hadoop facade providing access to Ignite Hadoop features.
- */
-public interface GridHadoop {
-    /**
-     * Gets Hadoop module configuration.
-     *
-     * @return Hadoop module configuration.
-     */
-    public GridHadoopConfiguration configuration();
-
-    /**
-     * Generate next job ID.
-     *
-     * @return Next job ID.
-     */
-    public GridHadoopJobId nextJobId();
-
-    /**
-     * Submits job to job tracker.
-     *
-     * @param jobId Job ID to submit.
-     * @param jobInfo Job info to submit.
-     * @return Execution future.
-     */
-    public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
-
-    /**
-     * Gets Hadoop job execution status.
-     *
-     * @param jobId Job ID to get status for.
-     * @return Job execution status or {@code null} in case job with the given ID is not found.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Returns job counters.
-     *
-     * @param jobId Job ID to get counters for.
-     * @return Job counters object.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Gets Hadoop finish future for particular job.
-     *
-     * @param jobId Job ID.
-     * @return Job finish future or {@code null} in case job with the given ID is not found.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Kills job.
-     *
-     * @param jobId Job ID.
-     * @return {@code True} if job was killed.
-     * @throws IgniteCheckedException If failed.
-     */
-    public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopConfiguration.java
deleted file mode 100644
index f66b95a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopConfiguration.java
+++ /dev/null
@@ -1,172 +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.internal.*;
-
-/**
- * Hadoop configuration.
- */
-public class GridHadoopConfiguration {
-    /** Default finished job info time-to-live. */
-    public static final long DFLT_FINISHED_JOB_INFO_TTL = 10_000;
-
-    /** Default value for external execution flag. */
-    public static final boolean DFLT_EXTERNAL_EXECUTION = false;
-
-    /** Default value for the max parallel tasks. */
-    public static final int DFLT_MAX_PARALLEL_TASKS = Runtime.getRuntime().availableProcessors();
-
-    /** Default value for the max task queue size. */
-    public static final int DFLT_MAX_TASK_QUEUE_SIZE = 1000;
-
-    /** Map reduce planner. */
-    private GridHadoopMapReducePlanner planner;
-
-    /** */
-    private boolean extExecution = DFLT_EXTERNAL_EXECUTION;
-
-    /** Finished job info TTL. */
-    private long finishedJobInfoTtl = DFLT_FINISHED_JOB_INFO_TTL;
-
-    /** */
-    private int maxParallelTasks = DFLT_MAX_PARALLEL_TASKS;
-
-    /** */
-    private int maxTaskQueueSize = DFLT_MAX_TASK_QUEUE_SIZE;
-
-    /**
-     * Default constructor.
-     */
-    public GridHadoopConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * Copy constructor.
-     *
-     * @param cfg Configuration to copy.
-     */
-    public GridHadoopConfiguration(GridHadoopConfiguration cfg) {
-        // Preserve alphabetic order.
-        extExecution = cfg.isExternalExecution();
-        finishedJobInfoTtl = cfg.getFinishedJobInfoTtl();
-        planner = cfg.getMapReducePlanner();
-        maxParallelTasks = cfg.getMaxParallelTasks();
-        maxTaskQueueSize = cfg.getMaxTaskQueueSize();
-    }
-
-    /**
-     * Gets max number of local tasks that may be executed in parallel.
-     *
-     * @return Max number of local tasks that may be executed in parallel.
-     */
-    public int getMaxParallelTasks() {
-        return maxParallelTasks;
-    }
-
-    /**
-     * Sets max number of local tasks that may be executed in parallel.
-     *
-     * @param maxParallelTasks Max number of local tasks that may be executed in parallel.
-     */
-    public void setMaxParallelTasks(int maxParallelTasks) {
-        this.maxParallelTasks = maxParallelTasks;
-    }
-
-    /**
-     * Gets max task queue size.
-     *
-     * @return Max task queue size.
-     */
-    public int getMaxTaskQueueSize() {
-        return maxTaskQueueSize;
-    }
-
-    /**
-     * Sets max task queue size.
-     *
-     * @param maxTaskQueueSize Max task queue size.
-     */
-    public void setMaxTaskQueueSize(int maxTaskQueueSize) {
-        this.maxTaskQueueSize = maxTaskQueueSize;
-    }
-
-    /**
-     * Gets finished job info time-to-live in milliseconds.
-     *
-     * @return Finished job info time-to-live.
-     */
-    public long getFinishedJobInfoTtl() {
-        return finishedJobInfoTtl;
-    }
-
-    /**
-     * Sets finished job info time-to-live.
-     *
-     * @param finishedJobInfoTtl Finished job info time-to-live.
-     */
-    public void setFinishedJobInfoTtl(long finishedJobInfoTtl) {
-        this.finishedJobInfoTtl = finishedJobInfoTtl;
-    }
-
-    /**
-     * Gets external task execution flag. If {@code true}, hadoop job tasks will be executed in an external
-     * (relative to node) process.
-     *
-     * @return {@code True} if external execution.
-     */
-    public boolean isExternalExecution() {
-        return extExecution;
-    }
-
-    /**
-     * Sets external task execution flag.
-     *
-     * @param extExecution {@code True} if tasks should be executed in an external process.
-     * @see #isExternalExecution()
-     */
-    public void setExternalExecution(boolean extExecution) {
-        this.extExecution = extExecution;
-    }
-
-    /**
-     * Gets Hadoop map-reduce planner, a component which defines job execution plan based on job
-     * configuration and current grid topology.
-     *
-     * @return Map-reduce planner.
-     */
-    public GridHadoopMapReducePlanner getMapReducePlanner() {
-        return planner;
-    }
-
-    /**
-     * Sets Hadoop map-reduce planner, a component which defines job execution plan based on job
-     * configuration and current grid topology.
-     *
-     * @param planner Map-reduce planner.
-     */
-    public void setMapReducePlanner(GridHadoopMapReducePlanner planner) {
-        this.planner = planner;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopConfiguration.class, this, super.toString());
-    }
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileBlock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileBlock.java
deleted file mode 100644
index fae111a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileBlock.java
+++ /dev/null
@@ -1,162 +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.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-
-/**
- * Hadoop file block.
- */
-public class GridHadoopFileBlock extends GridHadoopInputSplit {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    @GridToStringInclude
-    protected URI file;
-
-    /** */
-    @GridToStringInclude
-    protected long start;
-
-    /** */
-    @GridToStringInclude
-    protected long len;
-
-    /**
-     * Creates new file block.
-     */
-    public GridHadoopFileBlock() {
-        // No-op.
-    }
-
-    /**
-     * Creates new file block.
-     *
-     * @param hosts List of hosts where the block resides.
-     * @param file File URI.
-     * @param start Start position of the block in the file.
-     * @param len Length of the block.
-     */
-    public GridHadoopFileBlock(String[] hosts, URI file, long start, long len) {
-        A.notNull(hosts, "hosts", file, "file");
-
-        this.hosts = hosts;
-        this.file = file;
-        this.start = start;
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(file());
-        out.writeLong(start());
-        out.writeLong(length());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        file = (URI)in.readObject();
-        start = in.readLong();
-        len = in.readLong();
-    }
-
-    /**
-     * @return Length.
-     */
-    public long length() {
-        return len;
-    }
-
-    /**
-     * @param len New length.
-     */
-    public void length(long len) {
-        this.len = len;
-    }
-
-    /**
-     * @return Start.
-     */
-    public long start() {
-        return start;
-    }
-
-    /**
-     * @param start New start.
-     */
-    public void start(long start) {
-        this.start = start;
-    }
-
-    /**
-     * @return File.
-     */
-    public URI file() {
-        return file;
-    }
-
-    /**
-     * @param file New file.
-     */
-    public void file(URI file) {
-        this.file = file;
-    }
-
-    /**
-     * @param hosts New hosts.
-     */
-    public void hosts(String[] hosts) {
-        A.notNull(hosts, "hosts");
-
-        this.hosts = hosts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof GridHadoopFileBlock))
-            return false;
-
-        GridHadoopFileBlock that = (GridHadoopFileBlock)o;
-
-        return len == that.len && start == that.start && file.equals(that.file);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = file.hashCode();
-
-        res = 31 * res + (int)(start ^ (start >>> 32));
-        res = 31 * res + (int)(len ^ (len >>> 32));
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    public String toString() {
-        return S.toString(GridHadoopFileBlock.class, this, "hosts", Arrays.toString(hosts));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopInputSplit.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopInputSplit.java
deleted file mode 100644
index e68a6f5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopInputSplit.java
+++ /dev/null
@@ -1,54 +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 java.io.*;
-
-/**
- * Abstract fragment of an input data source.
- */
-public abstract class GridHadoopInputSplit implements Externalizable {
-    /** */
-    protected String[] hosts;
-
-    /**
-     * Array of hosts where this input split resides.
-     *
-     * @return Hosts.
-     */
-    public String[] hosts() {
-        assert hosts != null;
-
-        return hosts;
-    }
-
-    /**
-     * This method must be implemented for purpose of internal implementation.
-     *
-     * @param obj Another object.
-     * @return {@code true} If objects are equal.
-     */
-    @Override public abstract boolean equals(Object obj);
-
-    /**
-     * This method must be implemented for purpose of internal implementation.
-     *
-     * @return Hash code of the object.
-     */
-    @Override public abstract int hashCode();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJob.java
deleted file mode 100644
index f7ea105..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJob.java
+++ /dev/null
@@ -1,102 +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 java.util.*;
-
-/**
- * Hadoop job.
- */
-public interface GridHadoopJob {
-    /**
-     * Gets job ID.
-     *
-     * @return Job ID.
-     */
-    public GridHadoopJobId id();
-
-    /**
-     * Gets job information.
-     *
-     * @return Job information.
-     */
-    public GridHadoopJobInfo info();
-
-    /**
-     * Gets collection of input splits for this job.
-     *
-     * @return Input splits.
-     */
-    public Collection<GridHadoopInputSplit> input() throws IgniteCheckedException;
-
-    /**
-     * Returns context for task execution.
-     *
-     * @param info Task info.
-     * @return Task Context.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Does all the needed initialization for the job. Will be called on each node where tasks for this job must
-     * be executed.
-     * <p>
-     * If job is running in external mode this method will be called on instance in Ignite node with parameter
-     * {@code false} and on instance in external process with parameter {@code true}.
-     *
-     * @param external If {@code true} then this job instance resides in external process.
-     * @param locNodeId Local node ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException;
-
-    /**
-     * Release all the resources.
-     * <p>
-     * If job is running in external mode this method will be called on instance in Ignite node with parameter
-     * {@code false} and on instance in external process with parameter {@code true}.
-     *
-     * @param external If {@code true} then this job instance resides in external process.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void dispose(boolean external) throws IgniteCheckedException;
-
-    /**
-     * Prepare local environment for the task.
-     *
-     * @param info Task info.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Cleans up local environment of the task.
-     *
-     * @param info Task info.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Cleans up the job staging directory.
-     */
-    void cleanupStagingDirectory();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobId.java
deleted file mode 100644
index ffc2057..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobId.java
+++ /dev/null
@@ -1,103 +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.processors.cache.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Job ID.
- */
-public class GridHadoopJobId implements GridCacheInternal, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private UUID nodeId;
-
-    /** */
-    private int jobId;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public GridHadoopJobId() {
-        // No-op.
-    }
-
-    /**
-     * @param nodeId Node ID.
-     * @param jobId Job ID.
-     */
-    public GridHadoopJobId(UUID nodeId, int jobId) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-    }
-
-    public UUID globalId() {
-        return nodeId;
-    }
-
-    public int localId() {
-        return jobId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeUuid(out, nodeId);
-        out.writeInt(jobId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        nodeId = U.readUuid(in);
-        jobId = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GridHadoopJobId that = (GridHadoopJobId) o;
-
-        if (jobId != that.jobId)
-            return false;
-
-        if (!nodeId.equals(that.nodeId))
-            return false;
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return 31 * nodeId.hashCode() + jobId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return nodeId + "_" + jobId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobInfo.java
deleted file mode 100644
index 9a891f4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobInfo.java
+++ /dev/null
@@ -1,83 +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.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Compact job description.
- */
-public interface GridHadoopJobInfo extends Serializable {
-    /**
-     * Gets optional configuration property for the job.
-     *
-     * @param name Property name.
-     * @return Value or {@code null} if none.
-     */
-    @Nullable public String property(String name);
-
-    /**
-     * Checks whether job has combiner.
-     *
-     * @return {@code true} If job has combiner.
-     */
-    public boolean hasCombiner();
-
-    /**
-     * Checks whether job has reducer.
-     * Actual number of reducers will be in {@link GridHadoopMapReducePlan#reducers()}.
-     *
-     * @return Number of reducer.
-     */
-    public boolean hasReducer();
-
-    /**
-     * Creates new job instance for the given ID.
-     * {@link GridHadoopJobInfo} is reusable for multiple jobs while {@link GridHadoopJob} is for one job execution.
-     * This method will be called once for the same ID on one node, though it can be called on the same host
-     * multiple times from different processes (in case of multiple nodes on the same host or external execution).
-     *
-     * @param jobId Job ID.
-     * @param log Logger.
-     * @return Job.
-     * @throws IgniteCheckedException If failed.
-     */
-    GridHadoopJob createJob(GridHadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
-
-    /**
-     * @return Number of reducers configured for job.
-     */
-    public int reducers();
-
-    /**
-     * Gets job name.
-     *
-     * @return Job name.
-     */
-    public String jobName();
-
-    /**
-     * Gets user name.
-     *
-     * @return User name.
-     */
-    public String user();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobPhase.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobPhase.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobPhase.java
deleted file mode 100644
index cc122bb..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobPhase.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;
-
-/**
- * Job run phase.
- */
-public enum GridHadoopJobPhase {
-    /** Job is running setup task. */
-    PHASE_SETUP,
-
-    /** Job is running map and combine tasks. */
-    PHASE_MAP,
-
-    /** Job has finished all map tasks and running reduce tasks. */
-    PHASE_REDUCE,
-
-    /** Job is stopping due to exception during any of the phases. */
-    PHASE_CANCELLING,
-
-    /** Job has finished execution. */
-    PHASE_COMPLETE
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobProperty.java
deleted file mode 100644
index 0ece051..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobProperty.java
+++ /dev/null
@@ -1,138 +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.jetbrains.annotations.*;
-
-/**
- * Enumeration of optional properties supported by Ignite for Apache Hadoop.
- */
-public enum GridHadoopJobProperty {
-    /**
-     * Initial size for hashmap which stores output of mapper and will be used as input of combiner.
-     * <p>
-     * Setting it right allows to avoid rehashing.
-     */
-    COMBINER_HASHMAP_SIZE,
-
-    /**
-     * Initial size for hashmap which stores output of mapper or combiner and will be used as input of reducer.
-     * <p>
-     * Setting it right allows to avoid rehashing.
-     */
-    PARTITION_HASHMAP_SIZE,
-
-    /**
-     * Specifies number of concurrently running mappers for external execution mode.
-     * <p>
-     * If not specified, defaults to {@code Runtime.getRuntime().availableProcessors()}.
-     */
-    EXTERNAL_CONCURRENT_MAPPERS,
-
-    /**
-     * Specifies number of concurrently running reducers for external execution mode.
-     * <p>
-     * If not specified, defaults to {@code Runtime.getRuntime().availableProcessors()}.
-     */
-    EXTERNAL_CONCURRENT_REDUCERS,
-
-    /**
-     * Delay in milliseconds after which Ignite server will reply job status.
-     */
-    JOB_STATUS_POLL_DELAY,
-
-    /**
-     * Size in bytes of single memory page which will be allocated for data structures in shuffle.
-     * <p>
-     * By default is {@code 32 * 1024}.
-     */
-    SHUFFLE_OFFHEAP_PAGE_SIZE,
-
-    /**
-     * If set to {@code true} then input for combiner will not be sorted by key.
-     * Internally hash-map will be used instead of sorted one, so {@link Object#equals(Object)}
-     * and {@link Object#hashCode()} methods of key must be implemented consistently with
-     * comparator for that type. Grouping comparator is not supported if this setting is {@code true}.
-     * <p>
-     * By default is {@code false}.
-     */
-    SHUFFLE_COMBINER_NO_SORTING,
-
-    /**
-     * If set to {@code true} then input for reducer will not be sorted by key.
-     * Internally hash-map will be used instead of sorted one, so {@link Object#equals(Object)}
-     * and {@link Object#hashCode()} methods of key must be implemented consistently with
-     * comparator for that type. Grouping comparator is not supported if this setting is {@code true}.
-     * <p>
-     * By default is {@code false}.
-     */
-    SHUFFLE_REDUCER_NO_SORTING;
-
-    /** */
-    private final String ptyName;
-
-    /**
-     *
-     */
-    GridHadoopJobProperty() {
-        ptyName = "ignite." + name().toLowerCase().replace('_', '.');
-    }
-
-    /**
-     * @return Property name.
-     */
-    public String propertyName() {
-        return ptyName;
-    }
-
-    /**
-     * @param jobInfo Job info.
-     * @param pty Property.
-     * @param dflt Default value.
-     * @return Property value.
-     */
-    public static String get(GridHadoopJobInfo jobInfo, GridHadoopJobProperty pty, @Nullable String dflt) {
-        String res = jobInfo.property(pty.propertyName());
-
-        return res == null ? dflt : res;
-    }
-
-    /**
-     * @param jobInfo Job info.
-     * @param pty Property.
-     * @param dflt Default value.
-     * @return Property value.
-     */
-    public static int get(GridHadoopJobInfo jobInfo, GridHadoopJobProperty pty, int dflt) {
-        String res = jobInfo.property(pty.propertyName());
-
-        return res == null ? dflt : Integer.parseInt(res);
-    }
-
-    /**
-     * @param jobInfo Job info.
-     * @param pty Property.
-     * @param dflt Default value.
-     * @return Property value.
-     */
-    public static boolean get(GridHadoopJobInfo jobInfo, GridHadoopJobProperty pty, boolean dflt) {
-        String res = jobInfo.property(pty.propertyName());
-
-        return res == null ? dflt : Boolean.parseBoolean(res);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobStatus.java
deleted file mode 100644
index 02ea883..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobStatus.java
+++ /dev/null
@@ -1,207 +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.internal.*;
-
-import java.io.*;
-
-/**
- * Hadoop job status.
- */
-public class GridHadoopJobStatus implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Job ID. */
-    private GridHadoopJobId jobId;
-
-    /** Job name. */
-    private String jobName;
-
-    /** User. */
-    private String usr;
-
-    /** Pending mappers count. */
-    private int pendingMapperCnt;
-
-    /** Pending reducers count. */
-    private int pendingReducerCnt;
-
-    /** Total mappers count. */
-    private int totalMapperCnt;
-
-    /** Total reducers count. */
-    private int totalReducerCnt;
-    /** Phase. */
-    private GridHadoopJobPhase jobPhase;
-
-    /** */
-    private boolean failed;
-
-    /** Version. */
-    private long ver;
-
-    /**
-     * {@link Externalizable} support.
-     */
-    public GridHadoopJobStatus() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param jobId Job ID.
-     * @param jobName Job name.
-     * @param usr User.
-     * @param pendingMapperCnt Pending mappers count.
-     * @param pendingReducerCnt Pending reducers count.
-     * @param totalMapperCnt Total mappers count.
-     * @param totalReducerCnt Total reducers count.
-     * @param jobPhase Job phase.
-     * @param failed Failed.
-     * @param ver Version.
-     */
-    public GridHadoopJobStatus(
-        GridHadoopJobId jobId,
-        String jobName,
-        String usr,
-        int pendingMapperCnt,
-        int pendingReducerCnt,
-        int totalMapperCnt,
-        int totalReducerCnt,
-        GridHadoopJobPhase jobPhase,
-        boolean failed,
-        long ver
-    ) {
-        this.jobId = jobId;
-        this.jobName = jobName;
-        this.usr = usr;
-        this.pendingMapperCnt = pendingMapperCnt;
-        this.pendingReducerCnt = pendingReducerCnt;
-        this.totalMapperCnt = totalMapperCnt;
-        this.totalReducerCnt = totalReducerCnt;
-        this.jobPhase = jobPhase;
-        this.failed = failed;
-        this.ver = ver;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @return Job name.
-     */
-    public String jobName() {
-        return jobName;
-    }
-
-    /**
-     * @return User.
-     */
-    public String user() {
-        return usr;
-    }
-
-    /**
-     * @return Pending mappers count.
-     */
-    public int pendingMapperCnt() {
-        return pendingMapperCnt;
-    }
-
-    /**
-     * @return Pending reducers count.
-     */
-    public int pendingReducerCnt() {
-        return pendingReducerCnt;
-    }
-
-    /**
-     * @return Total mappers count.
-     */
-    public int totalMapperCnt() {
-        return totalMapperCnt;
-    }
-
-    /**
-     * @return Total reducers count.
-     */
-    public int totalReducerCnt() {
-        return totalReducerCnt;
-    }
-
-    /**
-     * @return Version.
-     */
-    public long version() {
-        return ver;
-    }
-
-    /**
-     * @return Job phase.
-     */
-    public GridHadoopJobPhase jobPhase() {
-        return jobPhase;
-    }
-
-    /**
-     * @return {@code true} If the job failed.
-     */
-    public boolean isFailed() {
-        return failed;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopJobStatus.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(jobId);
-        U.writeString(out, jobName);
-        U.writeString(out, usr);
-        out.writeInt(pendingMapperCnt);
-        out.writeInt(pendingReducerCnt);
-        out.writeInt(totalMapperCnt);
-        out.writeInt(totalReducerCnt);
-        out.writeObject(jobPhase);
-        out.writeBoolean(failed);
-        out.writeLong(ver);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = (GridHadoopJobId)in.readObject();
-        jobName = U.readString(in);
-        usr = U.readString(in);
-        pendingMapperCnt = in.readInt();
-        pendingReducerCnt = in.readInt();
-        totalMapperCnt = in.readInt();
-        totalReducerCnt = in.readInt();
-        jobPhase = (GridHadoopJobPhase)in.readObject();
-        failed = in.readBoolean();
-        ver = in.readLong();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReducePlan.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReducePlan.java
deleted file mode 100644
index 2fd5160..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReducePlan.java
+++ /dev/null
@@ -1,80 +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.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Map-reduce job execution plan.
- */
-public interface GridHadoopMapReducePlan extends Serializable {
-    /**
-     * Gets collection of file blocks for which mappers should be executed.
-     *
-     * @param nodeId Node ID to check.
-     * @return Collection of file blocks or {@code null} if no mappers should be executed on given node.
-     */
-    @Nullable public Collection<GridHadoopInputSplit> mappers(UUID nodeId);
-
-    /**
-     * Gets reducer IDs that should be started on given node.
-     *
-     * @param nodeId Node ID to check.
-     * @return Array of reducer IDs.
-     */
-    @Nullable public int[] reducers(UUID nodeId);
-
-    /**
-     * Gets collection of all node IDs involved in map part of job execution.
-     *
-     * @return Collection of node IDs.
-     */
-    public Collection<UUID> mapperNodeIds();
-
-    /**
-     * Gets collection of all node IDs involved in reduce part of job execution.
-     *
-     * @return Collection of node IDs.
-     */
-    public Collection<UUID> reducerNodeIds();
-
-    /**
-     * Gets overall number of mappers for the job.
-     *
-     * @return Number of mappers.
-     */
-    public int mappers();
-
-    /**
-     * Gets overall number of reducers for the job.
-     *
-     * @return Number of reducers.
-     */
-    public int reducers();
-
-    /**
-     * Gets node ID for reducer.
-     *
-     * @param reducer Reducer.
-     * @return Node ID.
-     */
-    public UUID nodeForReducer(int reducer);
-}


[51/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 0000000,39f42b2..30133f5
mode 000000,100644..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
@@@ -1,0 -1,1626 +1,1626 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.counter.HadoopCounters;
+ 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.HadoopJobPhase.*;
+ import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.*;
+ import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.*;
+ 
+ /**
+  * Hadoop job tracker.
+  */
+ public class HadoopJobTracker extends HadoopComponent {
+     /** */
+     private final GridMutex mux = new GridMutex();
+ 
+     /** */
+     private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> jobMetaPrj;
+ 
+     /** Projection with expiry policy for finished job updates. */
+     private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> finishedJobMetaPrj;
+ 
+     /** Map-reduce execution planner. */
+     @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+     private HadoopMapReducePlanner mrPlanner;
+ 
+     /** All the known jobs. */
 -    private final ConcurrentMap<HadoopJobId, GridFutureAdapterEx<HadoopJob>> jobs = new ConcurrentHashMap8<>();
++    private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJob>> jobs = new ConcurrentHashMap8<>();
+ 
+     /** Locally active jobs. */
+     private final ConcurrentMap<HadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
+ 
+     /** Locally requested finish futures. */
+     private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJobId>> 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<HadoopJobId, HadoopJobMetadata> jobMetaCache() {
+         GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> 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<HadoopJobId, HadoopJobMetadata>)
+                         sysCache.projection(HadoopJobId.class, HadoopJobMetadata.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<HadoopJobId, HadoopJobMetadata> finishedJobMetaCache() {
+         GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> 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<HadoopJobId, HadoopJobMetadata>() {
+                 @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends HadoopJobId,
+                     ? extends HadoopJobMetadata>> 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<HadoopJobId> 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<HadoopJobId> submit(HadoopJobId jobId, HadoopJobInfo info) {
+         if (!busyLock.tryReadLock()) {
 -            return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to execute map-reduce job " +
++            return new GridFinishedFuture<>(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);
+ 
+             HadoopJob job = job(jobId, info);
+ 
+             HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
+ 
+             HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info);
+ 
+             meta.mapReducePlan(mrPlan);
+ 
+             meta.pendingSplits(allSplits(mrPlan));
+             meta.pendingReducers(allReducers(mrPlan));
+ 
+             GridFutureAdapter<HadoopJobId> completeFut = new GridFutureAdapter<>();
+ 
+             GridFutureAdapter<HadoopJobId> 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();
+ 
+             HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.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);
++            return new GridFinishedFuture<>(e);
+         }
+         finally {
+             busyLock.readUnlock();
+         }
+     }
+ 
+     /**
+      * Convert Hadoop job metadata to job status.
+      *
+      * @param meta Metadata.
+      * @return Status.
+      */
+     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+     public static HadoopJobStatus status(HadoopJobMetadata meta) {
+         HadoopJobInfo jobInfo = meta.jobInfo();
+ 
+         return new HadoopJobStatus(
+             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 HadoopJobStatus status(HadoopJobId jobId) throws IgniteCheckedException {
+         if (!busyLock.tryReadLock())
+             return null; // Grid is stopping.
+ 
+         try {
+             HadoopJobMetadata 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(HadoopJobId jobId) throws IgniteCheckedException {
+         if (!busyLock.tryReadLock())
+             return null; // Grid is stopping.
+ 
+         try {
+             HadoopJobMetadata 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());
++                return new GridFinishedFuture<>(jobId);
+             }
+ 
+             GridFutureAdapter<HadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId,
+                 new GridFutureAdapter<HadoopJobId>());
+ 
+             // 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 HadoopMapReducePlan plan(HadoopJobId jobId) throws IgniteCheckedException {
+         if (!busyLock.tryReadLock())
+             return null;
+ 
+         try {
+             HadoopJobMetadata 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(HadoopTaskInfo info, HadoopTaskStatus 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<HadoopJobId, HadoopJobMetadata> 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(HadoopJobId jobId, EntryProcessor<HadoopJobId, HadoopJobMetadata, 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(HadoopJobId jobId, Collection<Integer> reducers,
+         HadoopProcessDescriptor 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<HadoopInputSplit, Integer> allSplits(HadoopMapReducePlan plan) {
+         Map<HadoopInputSplit, Integer> res = new HashMap<>();
+ 
+         int taskNum = 0;
+ 
+         for (UUID nodeId : plan.mapperNodeIds()) {
+             for (HadoopInputSplit 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(HadoopMapReducePlan 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()) {
+                 HadoopJobMetadata meta = (HadoopJobMetadata)metaObj;
+ 
+                 HadoopJobId jobId = meta.jobId();
+ 
+                 HadoopMapReducePlan plan = meta.mapReducePlan();
+ 
+                 HadoopJobPhase phase = meta.phase();
+ 
+                 try {
+                     if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
+                         // Failover setup task.
+                         HadoopJob job = job(jobId, meta.jobInfo());
+ 
+                         Collection<HadoopTaskInfo> 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<HadoopInputSplit> cancelSplits = null;
+ 
+                         for (UUID nodeId : plan.mapperNodeIds()) {
+                             if (ctx.kernalContext().discovery().node(nodeId) == null) {
+                                 // Node has left the grid.
+                                 Collection<HadoopInputSplit> 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 HadoopJobId, ? extends HadoopJobMetadata>> updated)
+         throws IgniteCheckedException {
+         UUID locNodeId = ctx.localNodeId();
+ 
+         for (CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata> entry : updated) {
+             HadoopJobId jobId = entry.getKey();
+             HadoopJobMetadata 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(HadoopJobId jobId, HadoopMapReducePlan 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(HadoopJobId jobId, HadoopJobMetadata meta, UUID locNodeId)
+         throws IgniteCheckedException {
+         JobLocalState state = activeJobs.get(jobId);
+ 
+         HadoopJob job = job(jobId, meta.jobInfo());
+ 
+         HadoopMapReducePlan plan = meta.mapReducePlan();
+ 
+         switch (meta.phase()) {
+             case PHASE_SETUP: {
+                 if (ctx.jobUpdateLeader()) {
+                     Collection<HadoopTaskInfo> 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<HadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta);
+ 
+                 if (tasks != null)
+                     ctx.taskExecutor().run(job, tasks);
+ 
+                 break;
+             }
+ 
+             case PHASE_REDUCE: {
+                 if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) {
+                     HadoopTaskInfo info = new HadoopTaskInfo(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<HadoopTaskInfo> 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()) {
+                             HadoopTaskInfo info = new HadoopTaskInfo(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<HadoopInputSplit> cancelMappers = new ArrayList<>();
+                     Collection<Integer> cancelReducers = new ArrayList<>();
+ 
+                     Collection<HadoopInputSplit> mappers = plan.mappers(ctx.localNodeId());
+ 
+                     if (mappers != null) {
+                         for (HadoopInputSplit 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<HadoopJobId> 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);
+ 
+                             HadoopCounterWriter writer = (HadoopCounterWriter)cls.newInstance();
+ 
+                             HadoopCounters 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<HadoopTaskInfo> setupTask(HadoopJobId jobId) {
+         if (activeJobs.containsKey(jobId))
+             return null;
+         else {
+             initState(jobId);
+ 
+             return Collections.singleton(new HadoopTaskInfo(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<HadoopTaskInfo> mapperTasks(Iterable<HadoopInputSplit> mappers, HadoopJobMetadata meta) {
+         UUID locNodeId = ctx.localNodeId();
+         HadoopJobId jobId = meta.jobId();
+ 
+         JobLocalState state = activeJobs.get(jobId);
+ 
+         Collection<HadoopTaskInfo> tasks = null;
+ 
+         if (mappers != null) {
+             if (state == null)
+                 state = initState(jobId);
+ 
+             for (HadoopInputSplit split : mappers) {
+                 if (state.addMapper(split)) {
+                     if (log.isDebugEnabled())
+                         log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId +
+                             ", split=" + split + ']');
+ 
+                     HadoopTaskInfo taskInfo = new HadoopTaskInfo(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<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJob job) {
+         UUID locNodeId = ctx.localNodeId();
+         HadoopJobId jobId = job.id();
+ 
+         JobLocalState state = activeJobs.get(jobId);
+ 
+         Collection<HadoopTaskInfo> 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 + ']');
+ 
+                     HadoopTaskInfo taskInfo = new HadoopTaskInfo(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(HadoopJobId 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 HadoopJob job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException {
 -        GridFutureAdapterEx<HadoopJob> fut = jobs.get(jobId);
++        GridFutureAdapter<HadoopJob> fut = jobs.get(jobId);
+ 
 -        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapterEx<HadoopJob>())) != null)
++        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapter<HadoopJob>())) != null)
+             return fut.get();
+ 
+         fut = jobs.get(jobId);
+ 
+         HadoopJob job = null;
+ 
+         try {
+             if (jobInfo == null) {
+                 HadoopJobMetadata 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(HadoopJobId jobId) throws IgniteCheckedException {
+         if (!busyLock.tryReadLock())
+             return false; // Grid is stopping.
+ 
+         try {
+             HadoopJobMetadata 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 HadoopCounters jobCounters(HadoopJobId jobId) throws IgniteCheckedException {
+         if (!busyLock.tryReadLock())
+             return null;
+ 
+         try {
+             final HadoopJobMetadata 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<HadoopInputSplit> 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(HadoopInputSplit 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(HadoopInputSplit 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 HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
+             final HadoopJobId 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 HadoopTaskInfo taskInfo, HadoopTaskStatus status,
+             final StackedProcessor prev) {
+             final HadoopJobId 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(HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
+             HadoopJobId 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(HadoopTaskInfo taskInfo, HadoopTaskStatus status,
+             final StackedProcessor prev) {
+             final HadoopJobId 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 HadoopJobPhase phase;
+ 
+         /**
+          * @param prev Previous closure.
+          * @param phase Phase to update.
+          */
+         private UpdatePhaseProcessor(@Nullable StackedProcessor prev, HadoopJobPhase phase) {
+             super(prev);
+ 
+             this.phase = phase;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata 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<HadoopInputSplit> splits;
+ 
+         /** Error. */
+         private final Throwable err;
+ 
+         /**
+          * @param prev Previous closure.
+          * @param split Mapper split to remove.
+          * @param err Error.
+          */
+         private RemoveMappersProcessor(@Nullable StackedProcessor prev, HadoopInputSplit 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<HadoopInputSplit> splits,
+             Throwable err) {
+             super(prev);
+ 
+             this.splits = splits;
+             this.err = err;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+             Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+ 
+             for (HadoopInputSplit 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(HadoopJobMetadata meta, HadoopJobMetadata 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 HadoopProcessDescriptor desc;
+ 
+         /**
+          * @param prev Previous closure.
+          * @param rdc Reducers to initialize.
+          * @param desc External process descriptor.
+          */
+         private InitializeReducersProcessor(@Nullable StackedProcessor prev,
+             Collection<Integer> rdc,
+             HadoopProcessDescriptor desc) {
+             super(prev);
+ 
+             assert !F.isEmpty(rdc);
+             assert desc != null;
+ 
+             this.rdc = rdc;
+             this.desc = desc;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+             Map<Integer, HadoopProcessDescriptor> oldMap = meta.reducersAddresses();
+ 
+             Map<Integer, HadoopProcessDescriptor> rdcMap = oldMap == null ?
+                 new HashMap<Integer, HadoopProcessDescriptor>() : 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<HadoopInputSplit> 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<HadoopInputSplit> 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<HadoopInputSplit> splits,
+             Collection<Integer> rdc) {
+             super(prev);
+ 
+             this.splits = splits;
+             this.rdc = rdc;
+             this.err = err;
+         }
+ 
+         /** {@inheritDoc} */
+         @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());
+ 
+             if (rdc != null)
+                 rdcCp.removeAll(rdc);
+ 
+             cp.pendingReducers(rdcCp);
+ 
+             Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+ 
+             if (splits != null) {
+                 for (HadoopInputSplit 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 HadoopCounters counters;
+ 
+         /**
+          * @param prev Previous closure.
+          * @param counters Task counters to add into job counters.
+          */
+         private IncrementCountersProcessor(@Nullable StackedProcessor prev, HadoopCounters counters) {
+             super(prev);
+ 
+             assert counters != null;
+ 
+             this.counters = counters;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+             HadoopCounters cntrs = new HadoopCountersImpl(cp.counters());
+ 
+             cntrs.merge(counters);
+ 
+             cp.counters(cntrs);
+         }
+     }
+ 
+     /**
+      * Abstract stacked closure.
+      */
+     private abstract static class StackedProcessor implements
+         EntryProcessor<HadoopJobId, HadoopJobMetadata, 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<HadoopJobId, HadoopJobMetadata> e, Object... args) {
+             HadoopJobMetadata val = apply(e.getValue());
+ 
+             if (val != null)
+                 e.setValue(val);
+             else
+                 e.remove();;
+ 
+             return null;
+         }
+ 
+         /**
+          * @param meta Old value.
+          * @return New value.
+          */
+         private HadoopJobMetadata apply(HadoopJobMetadata meta) {
+             if (meta == null)
+                 return null;
+ 
+             HadoopJobMetadata cp = prev != null ? prev.apply(meta) : new HadoopJobMetadata(meta);
+ 
+             update(meta, cp);
+ 
+             return cp;
+         }
+ 
+         /**
+          * Update given job metadata object.
+          *
+          * @param meta Initial job metadata.
+          * @param cp Copy.
+          */
+         protected abstract void update(HadoopJobMetadata meta, HadoopJobMetadata cp);
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 0000000,422d941..d173927
mode 000000,100644..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
@@@ -1,0 -1,256 +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<HadoopJobId, HadoopShuffleJob<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, (HadoopMessage)msg);
+                 }
+             });
+     }
+ 
+     /**
+      * Stops shuffle.
+      *
+      * @param cancel If should cancel all ongoing activities.
+      */
+     @Override public void stop(boolean cancel) {
+         for (HadoopShuffleJob 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 HadoopShuffleJob<UUID> newJob(HadoopJobId jobId) throws IgniteCheckedException {
+         HadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
+ 
+         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()];
+ 
+         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 HadoopShuffleJob<UUID> job(HadoopJobId jobId) throws IgniteCheckedException {
+         HadoopShuffleJob<UUID> res = jobs.get(jobId);
+ 
+         if (res == null) {
+             res = newJob(jobId);
+ 
+             HadoopShuffleJob<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(HadoopShuffleJob<UUID> shuffleJob) {
+         shuffleJob.startSending(ctx.kernalContext().gridName(),
+             new IgniteInClosure2X<UUID, HadoopShuffleMessage>() {
+                 @Override public void applyx(UUID dest, HadoopShuffleMessage 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, HadoopMessage msg) {
+         if (msg instanceof HadoopShuffleMessage) {
+             HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
+ 
+             try {
+                 job(m.jobId()).onShuffleMessage(m);
+             }
+             catch (IgniteCheckedException e) {
+                 U.error(log, "Message handling failed.", e);
+             }
+ 
+             try {
+                 // Reply with ack.
+                 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 HadoopShuffleAck) {
+             HadoopShuffleAck m = (HadoopShuffleAck)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 HadoopTaskOutput output(HadoopTaskContext taskCtx) throws IgniteCheckedException {
+         return job(taskCtx.taskInfo().jobId()).output(taskCtx);
+     }
+ 
+     /**
+      * @param taskCtx Task info.
+      * @return Input.
+      */
+     public HadoopTaskInput input(HadoopTaskContext taskCtx) throws IgniteCheckedException {
+         return job(taskCtx.taskInfo().jobId()).input(taskCtx);
+     }
+ 
+     /**
+      * @param jobId Job id.
+      */
+     public void jobFinished(HadoopJobId jobId) {
+         HadoopShuffleJob 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(HadoopJobId jobId) {
+         HadoopShuffleJob job = jobs.get(jobId);
+ 
+         if (job == null)
 -            return new GridFinishedFutureEx<>();
++            return new GridFinishedFuture<>();
+ 
+         try {
+             return job.flush();
+         }
+         catch (IgniteCheckedException e) {
 -            return new GridFinishedFutureEx<>(e);
++            return new GridFinishedFuture<>(e);
+         }
+     }
+ 
+     /**
+      * @return Memory.
+      */
+     public GridUnsafeMemory memory() {
+         return mem;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 0000000,7ae52df..1f92c66
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,593 +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.HadoopJobProperty.*;
+ 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 HadoopJob job;
+ 
+     /** */
+     private final GridUnsafeMemory mem;
+ 
+     /** */
+     private final boolean needPartitioner;
+ 
+     /** Collection of task contexts for each reduce task. */
+     private final Map<Integer, HadoopTaskContext> 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 =
++    protected ConcurrentMap<Long, IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapter<?>>> 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, HadoopJob 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) {
+                 HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.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;
+ 
+         HadoopTaskContext 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());
++        IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapter<?>> 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<>();
++        final GridFutureAdapter<?> fut = new GridFutureAdapter<>();
+ 
+         HadoopShuffleMessage msg = msgs[idx];
+ 
+         final long msgId = msg.id();
+ 
 -        IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>> old = sentMsgs.putIfAbsent(msgId,
 -            new IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>>(msg, fut));
++        IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapter<?>> old = sentMsgs.putIfAbsent(msgId,
++            new IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapter<?>>(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<>();
++            return new GridFinishedFuture<>();
+ 
+         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())
++        for (IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapter<?>> 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 HadoopTaskOutput output(HadoopTaskContext 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 HadoopTaskInput input(HadoopTaskContext 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 HadoopTaskInput() { // 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 HadoopTaskOutput {
+         /** */
+         private final HadoopTaskOutput[] adders = new HadoopTaskOutput[maps.length()];
+ 
+         /** */
+         private HadoopPartitioner partitioner;
+ 
+         /** */
+         private final HadoopTaskContext taskCtx;
+ 
+         /**
+          * Constructor.
+          * @param taskCtx Task context.
+          */
+         private PartitionedOutput(HadoopTaskContext 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);
+             }
+ 
+             HadoopTaskOutput 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 (HadoopTaskOutput adder : adders) {
+                 if (adder != null)
+                     adder.close();
+             }
+         }
+     }
+ }


[20/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..e95b8cb
--- /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.HadoopTaskType.*;
+
+/**
+ * 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 HadoopJob 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;
+
+                    HadoopTaskInfo 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 (HadoopTaskInfo 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 HadoopTaskInput createInput(HadoopTaskContext ctx)
+                                throws IgniteCheckedException {
+                                return shuffleJob.input(ctx);
+                            }
+
+                            @Override protected HadoopTaskOutput createOutput(HadoopTaskContext ctx)
+                                throws IgniteCheckedException {
+                                return shuffleJob.output(ctx);
+                            }
+                        });
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    for (HadoopTaskInfo 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) {
+        HadoopTaskInfo 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 HadoopTaskInfo 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();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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 b375b55..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(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 e3457a9..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).";
-
-    /** 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<GridHadoopMessage> srvLsnr =
-        new GridNioServerListenerAdapter<GridHadoopMessage>() {
-            @Override public void onConnected(GridNioSession ses) {
-                GridHadoopProcessDescriptor 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);
-
-                GridHadoopProcessDescriptor 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, GridHadoopMessage msg) {
-                notifyListener(ses.<GridHadoopProcessDescriptor>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 GridHadoopProcessDescriptor 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<GridHadoopMessage> 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 GridHadoopProcessDescriptor(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 GridHadoopProcessDescriptor 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<GridHadoopMessage> 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<GridHadoopMessage> srvr =
-                    GridNioServer.<GridHadoopMessage>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(GridHadoopProcessDescriptor desc, GridHadoopMessage 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(GridHadoopProcessDescriptor 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(GridHadoopProcessDescriptor 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(GridHadoopProcessDescriptor 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(GridHadoopProcessDescriptor 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(GridHadoopProcessDescriptor desc, GridHadoopMessage 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<GridHadoopMessage> 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 {
-            GridHadoopProcessDescriptor 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 GridHadoopMessage {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Node ID. */
-        private GridHadoopProcessDescriptor procDesc;
-
-        /** */
-        public ProcessHandshakeMessage() {
-            // No-op.
-        }
-
-        /**
-         * @param procDesc Process descriptor.
-         */
-        private ProcessHandshakeMessage(GridHadoopProcessDescriptor procDesc) {
-            this.procDesc = procDesc;
-        }
-
-        /**
-         * @return Process ID.
-         */
-        public GridHadoopProcessDescriptor 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 = (GridHadoopProcessDescriptor)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ProcessHandshakeMessage.class, this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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);
-    }
-}


[38/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
new file mode 100644
index 0000000..93c0df4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -0,0 +1,1237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1;
+
+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.hadoop.hdfs.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.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.net.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
+import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
+
+/**
+ * {@code IGFS} Hadoop 1.x file system driver over file system API. To use
+ * {@code IGFS} as Hadoop file system, you should configure this class
+ * in Hadoop's {@code core-site.xml} as follows:
+ * <pre name="code" class="xml">
+ *  &lt;property&gt;
+ *      &lt;name&gt;fs.default.name&lt;/name&gt;
+ *      &lt;value&gt;igfs://ipc&lt;/value&gt;
+ *  &lt;/property&gt;
+ *
+ *  &lt;property&gt;
+ *      &lt;name&gt;fs.igfs.impl&lt;/name&gt;
+ *      &lt;value&gt;org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem&lt;/value&gt;
+ *  &lt;/property&gt;
+ * </pre>
+ * You should also add Ignite JAR and all libraries to Hadoop classpath. To
+ * do this, add following lines to {@code conf/hadoop-env.sh} script in Hadoop
+ * distribution:
+ * <pre name="code" class="bash">
+ * export IGNITE_HOME=/path/to/Ignite/distribution
+ * export HADOOP_CLASSPATH=$IGNITE_HOME/ignite*.jar
+ *
+ * for f in $IGNITE_HOME/libs/*.jar; do
+ *  export HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$f;
+ * done
+ * </pre>
+ * <h1 class="header">Data vs Clients Nodes</h1>
+ * Hadoop needs to use its FileSystem remotely from client nodes as well as directly on
+ * data nodes. Client nodes are responsible for basic file system operations as well as
+ * accessing data nodes remotely. Usually, client nodes are started together
+ * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
+ * started together with Hadoop {@code task-tracker} processes.
+ * <p>
+ * For sample client and data node configuration refer to {@code config/hadoop/default-config-client.xml}
+ * and {@code config/hadoop/default-config.xml} configuration files in Ignite installation.
+ */
+public class IgniteHadoopFileSystem extends FileSystem {
+    /** Internal property to indicate management connection. */
+    public static final String IGFS_MANAGEMENT = "fs.igfs.management.connection";
+
+    /** Empty array of file block locations. */
+    private static final BlockLocation[] EMPTY_BLOCK_LOCATIONS = new BlockLocation[0];
+
+    /** Empty array of file statuses. */
+    public static final FileStatus[] EMPTY_FILE_STATUS = new FileStatus[0];
+
+    /** Ensures that close routine is invoked at most once. */
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /** Grid remote client. */
+    private HadoopIgfsWrapper rmtClient;
+
+    /** 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();
+        }
+    };
+
+    /** Default replication factor. */
+    private short dfltReplication;
+
+    /** Base file system uri. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private URI uri;
+
+    /** Authority. */
+    private String uriAuthority;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Secondary URI string. */
+    private URI secondaryUri;
+
+    /** IGFS mode resolver. */
+    private IgfsModeResolver modeRslvr;
+
+    /** Secondary file system instance. */
+    private FileSystem secondaryFs;
+
+    /** Management connection flag. */
+    private boolean mgmt;
+
+    /** Whether custom sequential reads before prefetch value is provided. */
+    private boolean seqReadsBeforePrefetchOverride;
+
+    /** IGFS group block size. */
+    private long igfsGrpBlockSize;
+
+    /** Flag that controls whether file writes should be colocated. */
+    private boolean colocateFileWrites;
+
+    /** Prefer local writes. */
+    private boolean preferLocFileWrites;
+
+    /** Custom-provided sequential reads before prefetch. */
+    private int seqReadsBeforePrefetch;
+
+    /** The cache was disabled when the instance was creating. */
+    private boolean cacheEnabled;
+
+    /** {@inheritDoc} */
+    @Override public URI getUri() {
+        if (uri == null)
+            throw new IllegalStateException("URI is null (was IgniteHadoopFileSystem properly initialized?).");
+
+        return uri;
+    }
+
+    /**
+     * Enter busy state.
+     *
+     * @throws IOException If file system is stopped.
+     */
+    private void enterBusy() throws IOException {
+        if (closeGuard.get())
+            throw new IOException("File system is stopped.");
+    }
+
+    /**
+     * Leave busy state.
+     */
+    private void leaveBusy() {
+        // No-op.
+    }
+
+    /**
+     * Public setter that can be used by direct users of FS or Visor.
+     *
+     * @param colocateFileWrites Whether all ongoing file writes should be colocated.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void colocateFileWrites(boolean colocateFileWrites) {
+        this.colocateFileWrites = colocateFileWrites;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public void initialize(URI name, Configuration cfg) throws IOException {
+        enterBusy();
+
+        try {
+            if (rmtClient != null)
+                throw new IOException("File system is already initialized: " + rmtClient);
+
+            A.notNull(name, "name");
+            A.notNull(cfg, "cfg");
+
+            super.initialize(name, cfg);
+
+            setConf(cfg);
+
+            String disableCacheName = String.format("fs.%s.impl.disable.cache", name.getScheme());
+
+            cacheEnabled = !cfg.getBoolean(disableCacheName, false);
+
+            mgmt = cfg.getBoolean(IGFS_MANAGEMENT, false);
+
+            if (!IGFS_SCHEME.equals(name.getScheme()))
+                throw new IOException("Illegal file system URI [expected=" + IGFS_SCHEME +
+                    "://[name]/[optional_path], actual=" + name + ']');
+
+            uri = name;
+
+            uriAuthority = uri.getAuthority();
+
+            setUser(cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+
+            // Override sequential reads before prefetch if needed.
+            seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
+
+            if (seqReadsBeforePrefetch > 0)
+                seqReadsBeforePrefetchOverride = true;
+
+            // In Ignite replication factor is controlled by data cache affinity.
+            // We use replication factor to force the whole file to be stored on local node.
+            dfltReplication = (short)cfg.getInt("dfs.replication", 3);
+
+            // Get file colocation control flag.
+            colocateFileWrites = parameter(cfg, PARAM_IGFS_COLOCATED_WRITES, uriAuthority, false);
+            preferLocFileWrites = cfg.getBoolean(PARAM_IGFS_PREFER_LOCAL_WRITES, false);
+
+            // Get log directory.
+            String logDirCfg = parameter(cfg, PARAM_IGFS_LOG_DIR, uriAuthority, DFLT_IGFS_LOG_DIR);
+
+            File logDirFile = U.resolveIgnitePath(logDirCfg);
+
+            String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
+
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+
+            // Handshake.
+            IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
+
+            igfsGrpBlockSize = handshake.blockSize();
+
+            IgfsPaths paths = handshake.secondaryPaths();
+
+            // Initialize client logger.
+            Boolean logEnabled = parameter(cfg, PARAM_IGFS_LOG_ENABLED, uriAuthority, false);
+
+            if (handshake.sampling() != null ? handshake.sampling() : logEnabled) {
+                // Initiate client logger.
+                if (logDir == null)
+                    throw new IOException("Failed to resolve log directory: " + logDirCfg);
+
+                Integer batchSize = parameter(cfg, PARAM_IGFS_LOG_BATCH_SIZE, uriAuthority, DFLT_IGFS_LOG_BATCH_SIZE);
+
+                clientLog = IgfsLogger.logger(uriAuthority, handshake.igfsName(), logDir, batchSize);
+            }
+            else
+                clientLog = IgfsLogger.disabledLogger();
+
+            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+
+            boolean initSecondary = paths.defaultMode() == PROXY;
+
+            if (!initSecondary && paths.pathModes() != null && !paths.pathModes().isEmpty()) {
+                for (T2<IgfsPath, IgfsMode> pathMode : paths.pathModes()) {
+                    IgfsMode mode = pathMode.getValue();
+
+                    if (mode == PROXY) {
+                        initSecondary = true;
+
+                        break;
+                    }
+                }
+            }
+
+            if (initSecondary) {
+                Map<String, String> props = paths.properties();
+
+                String secUri = props.get(SECONDARY_FS_URI);
+                String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
+
+                try {
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createFileSystem();
+                    secondaryUri = secProvider.uri();
+                }
+                catch (IOException e) {
+                    if (!mgmt)
+                        throw new IOException("Failed to connect to the secondary file system: " + secUri, e);
+                    else
+                        LOG.warn("Visor failed to create secondary file system (operations on paths with PROXY mode " +
+                            "will have no effect): " + e.getMessage());
+                }
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkPath(Path path) {
+        URI uri = path.toUri();
+
+        if (uri.isAbsolute()) {
+            if (!F.eq(uri.getScheme(), IGFS_SCHEME))
+                throw new InvalidPathException("Wrong path scheme [expected=" + IGFS_SCHEME + ", actual=" +
+                    uri.getAuthority() + ']');
+
+            if (!F.eq(uri.getAuthority(), uriAuthority))
+                throw new InvalidPathException("Wrong path authority [expected=" + uriAuthority + ", actual=" +
+                    uri.getAuthority() + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public short getDefaultReplication() {
+        return dfltReplication;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void finalize() throws Throwable {
+        super.finalize();
+
+        close0();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        if (cacheEnabled && get(getUri(), getConf()) == this)
+            return;
+
+        close0();
+    }
+
+    /**
+     * Closes file system.
+     *
+     * @throws IOException If failed.
+     */
+    private void close0() throws IOException {
+        if (closeGuard.compareAndSet(false, true)) {
+            if (LOG.isDebugEnabled())
+                LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
+
+            if (rmtClient == null)
+                return;
+
+            super.close();
+
+            rmtClient.close(false);
+
+            if (clientLog.isLogEnabled())
+                clientLog.close();
+
+            if (secondaryFs != null)
+                U.closeQuiet(secondaryFs);
+
+            // Reset initialized resources.
+            uri = null;
+            rmtClient = null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setTimes(Path p, long mtime, long atime) throws IOException {
+        enterBusy();
+
+        try {
+            A.notNull(p, "p");
+
+            if (mode(p) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    // No-op for management connection.
+                    return;
+                }
+
+                secondaryFs.setTimes(toSecondary(p), mtime, atime);
+            }
+            else {
+                IgfsPath path = convert(p);
+
+                rmtClient.setTimes(path, atime, mtime);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setPermission(Path p, FsPermission perm) throws IOException {
+        enterBusy();
+
+        try {
+            A.notNull(p, "p");
+
+            if (mode(p) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    // No-op for management connection.
+                    return;
+                }
+
+                secondaryFs.setPermission(toSecondary(p), perm);
+            }
+            else if (rmtClient.update(convert(p), permission(perm)) == null) {
+                throw new IOException("Failed to set file permission (file not found?)" +
+                    " [path=" + p + ", perm=" + perm + ']');
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setOwner(Path p, String username, String grpName) throws IOException {
+        A.notNull(p, "p");
+        A.notNull(username, "username");
+        A.notNull(grpName, "grpName");
+
+        enterBusy();
+
+        try {
+            if (mode(p) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    // No-op for management connection.
+                    return;
+                }
+
+                secondaryFs.setOwner(toSecondary(p), username, grpName);
+            }
+            else if (rmtClient.update(convert(p), F.asMap(PROP_USER_NAME, username, PROP_GROUP_NAME, grpName)) == null)
+                throw new IOException("Failed to set file permission (file not found?)" +
+                    " [path=" + p + ", userName=" + username + ", groupName=" + grpName + ']');
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FSDataInputStream open(Path f, int bufSize) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    throw new IOException("Failed to open file (secondary file system is not initialized): " + f);
+                }
+
+                FSDataInputStream is = secondaryFs.open(toSecondary(f), bufSize);
+
+                if (clientLog.isLogEnabled()) {
+                    // At this point we do not know file size, so we perform additional request to remote FS to get it.
+                    FileStatus status = secondaryFs.getFileStatus(toSecondary(f));
+
+                    long size = status != null ? status.getLen() : -1;
+
+                    long logId = IgfsLogger.nextId();
+
+                    clientLog.logOpen(logId, path, PROXY, bufSize, size);
+
+                    return new FSDataInputStream(new HadoopIgfsProxyInputStream(is, clientLog, logId));
+                }
+                else
+                    return is;
+            }
+            else {
+                HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
+                    rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
+
+                long logId = -1;
+
+                if (clientLog.isLogEnabled()) {
+                    logId = IgfsLogger.nextId();
+
+                    clientLog.logOpen(logId, path, mode, bufSize, stream.length());
+                }
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
+                        ", bufSize=" + bufSize + ']');
+
+                HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
+                    bufSize, LOG, clientLog, logId);
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
+
+                return new FSDataInputStream(igfsIn);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public FSDataOutputStream create(Path f, FsPermission perm, boolean overwrite, int bufSize,
+        short replication, long blockSize, Progressable progress) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        OutputStream out = null;
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opening output stream in create [thread=" + Thread.currentThread().getName() + "path=" +
+                    path + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ']');
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    throw new IOException("Failed to create file (secondary file system is not initialized): " + f);
+                }
+
+                FSDataOutputStream os =
+                    secondaryFs.create(toSecondary(f), perm, overwrite, bufSize, replication, blockSize, progress);
+
+                if (clientLog.isLogEnabled()) {
+                    long logId = IgfsLogger.nextId();
+
+                    clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
+
+                    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.
+                HadoopIgfsStreamDelegate stream = rmtClient.create(path, overwrite, colocateFileWrites,
+                    replication, blockSize, F.asMap(PROP_PERMISSION, toString(perm),
+                    PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites)));
+
+                assert stream != null;
+
+                long logId = -1;
+
+                if (clientLog.isLogEnabled()) {
+                    logId = IgfsLogger.nextId();
+
+                    clientLog.logCreate(logId, path, mode, overwrite, bufSize, replication, blockSize);
+                }
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
+
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
+                    logId);
+
+                bufSize = Math.max(64 * 1024, bufSize);
+
+                out = new BufferedOutputStream(igfsOut, bufSize);
+
+                FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
+
+                // Mark stream created successfully.
+                out = null;
+
+                return res;
+            }
+        }
+        finally {
+            // Close if failed during stream creation.
+            if (out != null)
+                U.closeQuiet(out);
+
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public FSDataOutputStream append(Path f, int bufSize, Progressable progress) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opening output stream in append [thread=" + Thread.currentThread().getName() +
+                    ", path=" + path + ", bufSize=" + bufSize + ']');
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    throw new IOException("Failed to append file (secondary file system is not initialized): " + f);
+                }
+
+                FSDataOutputStream os = secondaryFs.append(toSecondary(f), bufSize, progress);
+
+                if (clientLog.isLogEnabled()) {
+                    long logId = IgfsLogger.nextId();
+
+                    clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
+
+                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
+                }
+                else
+                    return os;
+            }
+            else {
+                HadoopIgfsStreamDelegate stream = rmtClient.append(path, false, null);
+
+                assert stream != null;
+
+                long logId = -1;
+
+                if (clientLog.isLogEnabled()) {
+                    logId = IgfsLogger.nextId();
+
+                    clientLog.logAppend(logId, path, mode, bufSize);
+                }
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
+
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
+                    logId);
+
+                bufSize = Math.max(64 * 1024, bufSize);
+
+                BufferedOutputStream out = new BufferedOutputStream(igfsOut, bufSize);
+
+                return new FSDataOutputStream(out, null, 0);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public boolean rename(Path src, Path dst) throws IOException {
+        A.notNull(src, "src");
+        A.notNull(dst, "dst");
+
+        enterBusy();
+
+        try {
+            IgfsPath srcPath = convert(src);
+            IgfsPath dstPath = convert(dst);
+            IgfsMode mode = mode(srcPath);
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    return false;
+                }
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logRename(srcPath, PROXY, dstPath);
+
+                return secondaryFs.rename(toSecondary(src), toSecondary(dst));
+            }
+            else {
+                // Will throw exception if failed.
+                rmtClient.rename(srcPath, dstPath);
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logRename(srcPath, mode, dstPath);
+
+                return true;
+            }
+        }
+        catch (IOException e) {
+            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
+            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
+                !X.hasCause(e.getCause(), IgfsException.class)))
+                throw e;
+            else
+                return false;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public boolean delete(Path f) throws IOException {
+        return delete(f, false);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public boolean delete(Path f, boolean recursive) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    return false;
+                }
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logDelete(path, PROXY, recursive);
+
+                return secondaryFs.delete(toSecondary(f), recursive);
+            }
+            else {
+                // Will throw exception if delete failed.
+                boolean res = rmtClient.delete(path, recursive);
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logDelete(path, mode, recursive);
+
+                return res;
+            }
+        }
+        catch (IOException e) {
+            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
+            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
+                !X.hasCause(e.getCause(), IgfsException.class)))
+                throw e;
+            else
+                return false;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus[] listStatus(Path f) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    return EMPTY_FILE_STATUS;
+                }
+
+                FileStatus[] arr = secondaryFs.listStatus(toSecondary(f));
+
+                if (arr == null)
+                    throw new FileNotFoundException("File " + f + " does not exist.");
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = toPrimary(arr[i]);
+
+                if (clientLog.isLogEnabled()) {
+                    String[] fileArr = new String[arr.length];
+
+                    for (int i = 0; i < arr.length; i++)
+                        fileArr[i] = arr[i].getPath().toString();
+
+                    clientLog.logListDirectory(path, PROXY, fileArr);
+                }
+
+                return arr;
+            }
+            else {
+                Collection<IgfsFile> list = rmtClient.listFiles(path);
+
+                if (list == null)
+                    throw new FileNotFoundException("File " + f + " does not exist.");
+
+                List<IgfsFile> files = new ArrayList<>(list);
+
+                FileStatus[] arr = new FileStatus[files.size()];
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = convert(files.get(i));
+
+                if (clientLog.isLogEnabled()) {
+                    String[] fileArr = new String[arr.length];
+
+                    for (int i = 0; i < arr.length; i++)
+                        fileArr[i] = arr[i].getPath().toString();
+
+                    clientLog.logListDirectory(path, mode, fileArr);
+                }
+
+                return arr;
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getHomeDirectory() {
+        Path path = new Path("/user/" + userName.get());
+
+        return path.makeQualified(getUri(), null);
+    }
+
+    /**
+     * Set user name and default working directory for current thread.
+     *
+     * @param userName User name.
+     */
+    public void setUser(String userName) {
+        this.userName.set(userName);
+
+        setWorkingDirectory(null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setWorkingDirectory(Path newPath) {
+        if (newPath == null) {
+            Path homeDir = getHomeDirectory();
+
+            if (secondaryFs != null)
+                secondaryFs.setWorkingDirectory(toSecondary(homeDir));
+
+            workingDir.set(homeDir);
+        }
+        else {
+            Path fixedNewPath = fixRelativePart(newPath);
+
+            String res = fixedNewPath.toUri().getPath();
+
+            if (!DFSUtil.isValidName(res))
+                throw new IllegalArgumentException("Invalid DFS directory name " + res);
+
+            if (secondaryFs != null)
+                secondaryFs.setWorkingDirectory(toSecondary(fixedNewPath));
+
+            workingDir.set(fixedNewPath);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getWorkingDirectory() {
+        return workingDir.get();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public boolean mkdirs(Path f, FsPermission perm) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = mode(path);
+
+            if (mode == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    return false;
+                }
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logMakeDirectory(path, PROXY);
+
+                return secondaryFs.mkdirs(toSecondary(f), perm);
+            }
+            else {
+                boolean mkdirRes = rmtClient.mkdirs(path, permission(perm));
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logMakeDirectory(path, mode);
+
+                return mkdirRes;
+            }
+        }
+        catch (IOException e) {
+            // Intentionally ignore IGFS exceptions here to follow Hadoop contract.
+            if (F.eq(IOException.class, e.getClass()) && (e.getCause() == null ||
+                !X.hasCause(e.getCause(), IgfsException.class)))
+                throw e;
+            else
+                return false;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus getFileStatus(Path f) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            if (mode(f) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    throw new IOException("Failed to get file status (secondary file system is not initialized): " + f);
+                }
+
+                return toPrimary(secondaryFs.getFileStatus(toSecondary(f)));
+            }
+            else {
+                IgfsFile info = rmtClient.info(convert(f));
+
+                if (info == null)
+                    throw new FileNotFoundException("File not found: " + f);
+
+                return convert(info);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ContentSummary getContentSummary(Path f) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            if (mode(f) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    throw new IOException("Failed to get content summary (secondary file system is not initialized): " +
+                        f);
+                }
+
+                return secondaryFs.getContentSummary(toSecondary(f));
+            }
+            else {
+                IgfsPathSummary sum = rmtClient.contentSummary(convert(f));
+
+                return new ContentSummary(sum.totalLength(), sum.filesCount(), sum.directoriesCount(),
+                    -1, sum.totalLength(), rmtClient.fsStatus().spaceTotal());
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public BlockLocation[] getFileBlockLocations(FileStatus status, long start, long len) throws IOException {
+        A.notNull(status, "status");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(status.getPath());
+
+            if (mode(status.getPath()) == PROXY) {
+                if (secondaryFs == null) {
+                    assert mgmt;
+
+                    return EMPTY_BLOCK_LOCATIONS;
+                }
+
+                Path secPath = toSecondary(status.getPath());
+
+                return secondaryFs.getFileBlockLocations(secondaryFs.getFileStatus(secPath), start, len);
+            }
+            else {
+                long now = System.currentTimeMillis();
+
+                List<IgfsBlockLocation> affinity = new ArrayList<>(rmtClient.affinity(path, start, len));
+
+                BlockLocation[] arr = new BlockLocation[affinity.size()];
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = convert(affinity.get(i));
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
+                        (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
+
+                return arr;
+            }
+        }
+        catch (FileNotFoundException ignored) {
+            return EMPTY_BLOCK_LOCATIONS;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public long getDefaultBlockSize() {
+        return igfsGrpBlockSize;
+    }
+
+    /**
+     * Resolve path mode.
+     *
+     * @param path HDFS path.
+     * @return Path mode.
+     */
+    public IgfsMode mode(Path path) {
+        return mode(convert(path));
+    }
+
+    /**
+     * Resolve path mode.
+     *
+     * @param path IGFS path.
+     * @return Path mode.
+     */
+    public IgfsMode mode(IgfsPath path) {
+        return modeRslvr.resolveMode(path);
+    }
+
+    /**
+     * Convert the given path to path acceptable by the primary file system.
+     *
+     * @param path Path.
+     * @return Primary file system path.
+     */
+    private Path toPrimary(Path path) {
+        return convertPath(path, uri);
+    }
+
+    /**
+     * Convert the given path to path acceptable by the secondary file system.
+     *
+     * @param path Path.
+     * @return Secondary file system path.
+     */
+    private Path toSecondary(Path path) {
+        assert secondaryFs != null;
+        assert secondaryUri != null;
+
+        return convertPath(path, secondaryUri);
+    }
+
+    /**
+     * Convert path using the given new URI.
+     *
+     * @param path Old path.
+     * @param newUri New URI.
+     * @return New path.
+     */
+    private Path convertPath(Path path, URI newUri) {
+        assert newUri != null;
+
+        if (path != null) {
+            URI pathUri = path.toUri();
+
+            try {
+                return new Path(new URI(pathUri.getScheme() != null ? newUri.getScheme() : null,
+                    pathUri.getAuthority() != null ? newUri.getAuthority() : null, pathUri.getPath(), null, null));
+            }
+            catch (URISyntaxException e) {
+                throw new IgniteException("Failed to construct secondary file system path from the primary file " +
+                    "system path: " + path, e);
+            }
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Convert a file status obtained from the secondary file system to a status of the primary file system.
+     *
+     * @param status Secondary file system status.
+     * @return Primary file system status.
+     */
+    @SuppressWarnings("deprecation")
+    private FileStatus toPrimary(FileStatus status) {
+        return status != null ? new FileStatus(status.getLen(), status.isDir(), status.getReplication(),
+            status.getBlockSize(), status.getModificationTime(), status.getAccessTime(), status.getPermission(),
+            status.getOwner(), status.getGroup(), toPrimary(status.getPath())) : null;
+    }
+
+    /**
+     * Convert IGFS path into Hadoop path.
+     *
+     * @param path IGFS path.
+     * @return Hadoop path.
+     */
+    private Path convert(IgfsPath path) {
+        return new Path(IGFS_SCHEME, uriAuthority, path.toString());
+    }
+
+    /**
+     * Convert Hadoop path into IGFS path.
+     *
+     * @param path Hadoop path.
+     * @return IGFS path.
+     */
+    @Nullable private IgfsPath convert(@Nullable Path path) {
+        if (path == null)
+            return null;
+
+        return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
+            new IgfsPath(convert(workingDir.get()), path.toUri().getPath());
+    }
+
+    /**
+     * Convert IGFS affinity block location into Hadoop affinity block location.
+     *
+     * @param block IGFS affinity block location.
+     * @return Hadoop affinity block location.
+     */
+    private BlockLocation convert(IgfsBlockLocation block) {
+        Collection<String> names = block.names();
+        Collection<String> hosts = block.hosts();
+
+        return new BlockLocation(
+            names.toArray(new String[names.size()]) /* hostname:portNumber of data nodes */,
+            hosts.toArray(new String[hosts.size()]) /* hostnames of data nodes */,
+            block.start(), block.length()
+        ) {
+            @Override public String toString() {
+                try {
+                    return "BlockLocation [offset=" + getOffset() + ", length=" + getLength() +
+                        ", hosts=" + Arrays.asList(getHosts()) + ", names=" + Arrays.asList(getNames()) + ']';
+                }
+                catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Convert IGFS file information into Hadoop file status.
+     *
+     * @param file IGFS file information.
+     * @return Hadoop file status.
+     */
+    @SuppressWarnings("deprecation")
+    private FileStatus convert(IgfsFile file) {
+        return new FileStatus(file.length(), file.isDirectory(), getDefaultReplication(),
+            file.groupBlockSize(), file.modificationTime(), file.accessTime(), permission(file),
+            file.property(PROP_USER_NAME, DFLT_USER_NAME), file.property(PROP_GROUP_NAME, "users"),
+            convert(file.path())) {
+            @Override public String toString() {
+                return "FileStatus [path=" + getPath() + ", isDir=" + isDir() + ", len=" + getLen() +
+                    ", mtime=" + getModificationTime() + ", atime=" + getAccessTime() + ']';
+            }
+        };
+    }
+
+    /**
+     * Convert Hadoop permission into IGFS file attribute.
+     *
+     * @param perm Hadoop permission.
+     * @return IGFS attributes.
+     */
+    private Map<String, String> permission(FsPermission perm) {
+        if (perm == null)
+            perm = FsPermission.getDefault();
+
+        return F.asMap(PROP_PERMISSION, toString(perm));
+    }
+
+    /**
+     * @param perm Permission.
+     * @return String.
+     */
+    private static String toString(FsPermission perm) {
+        return String.format("%04o", perm.toShort());
+    }
+
+    /**
+     * Convert IGFS file attributes into Hadoop permission.
+     *
+     * @param file File info.
+     * @return Hadoop permission.
+     */
+    private FsPermission permission(IgfsFile file) {
+        String perm = file.property(PROP_PERMISSION, null);
+
+        if (perm == null)
+            return FsPermission.getDefault();
+
+        try {
+            return new FsPermission((short)Integer.parseInt(perm, 8));
+        }
+        catch (NumberFormatException ignore) {
+            return FsPermission.getDefault();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteHadoopFileSystem.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/package.html b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/package.html
new file mode 100644
index 0000000..69801af
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/package.html
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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 Ignite Hadoop 1.x <code>FileSystem</code> implementation.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
new file mode 100644
index 0000000..70ad99f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -0,0 +1,982 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v2;
+
+import org.apache.commons.logging.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.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.net.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
+import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
+
+/**
+ * {@code IGFS} Hadoop 2.x file system driver over file system API. To use
+ * {@code IGFS} as Hadoop file system, you should configure this class
+ * in Hadoop's {@code core-site.xml} as follows:
+ * <pre name="code" class="xml">
+ *  &lt;property&gt;
+ *      &lt;name&gt;fs.default.name&lt;/name&gt;
+ *      &lt;value&gt;igfs://ipc&lt;/value&gt;
+ *  &lt;/property&gt;
+ *
+ *  &lt;property&gt;
+ *      &lt;name&gt;fs.igfs.impl&lt;/name&gt;
+ *      &lt;value&gt;org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem&lt;/value&gt;
+ *  &lt;/property&gt;
+ * </pre>
+ * You should also add Ignite JAR and all libraries to Hadoop classpath. To
+ * do this, add following lines to {@code conf/hadoop-env.sh} script in Hadoop
+ * distribution:
+ * <pre name="code" class="bash">
+ * export IGNITE_HOME=/path/to/Ignite/distribution
+ * export HADOOP_CLASSPATH=$IGNITE_HOME/ignite*.jar
+ *
+ * for f in $IGNITE_HOME/libs/*.jar; do
+ *  export HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$f;
+ * done
+ * </pre>
+ * <h1 class="header">Data vs Clients Nodes</h1>
+ * Hadoop needs to use its FileSystem remotely from client nodes as well as directly on
+ * data nodes. Client nodes are responsible for basic file system operations as well as
+ * accessing data nodes remotely. Usually, client nodes are started together
+ * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
+ * started together with Hadoop {@code task-tracker} processes.
+ * <p>
+ * For sample client and data node configuration refer to {@code config/hadoop/default-config-client.xml}
+ * and {@code config/hadoop/default-config.xml} configuration files in Ignite installation.
+ */
+public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closeable {
+    /** Logger. */
+    private static final Log LOG = LogFactory.getLog(IgniteHadoopFileSystem.class);
+
+    /** Ensures that close routine is invoked at most once. */
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /** Grid remote client. */
+    private HadoopIgfsWrapper rmtClient;
+
+    /** Working directory. */
+    private IgfsPath workingDir;
+
+    /** URI. */
+    private URI uri;
+
+    /** Authority. */
+    private String uriAuthority;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Server block size. */
+    private long grpBlockSize;
+
+    /** Default replication factor. */
+    private short dfltReplication;
+
+    /** Secondary URI string. */
+    private URI secondaryUri;
+
+    /** Mode resolver. */
+    private IgfsModeResolver modeRslvr;
+
+    /** Secondary file system instance. */
+    private AbstractFileSystem secondaryFs;
+
+    /** Whether custom sequential reads before prefetch value is provided. */
+    private boolean seqReadsBeforePrefetchOverride;
+
+    /** Custom-provided sequential reads before prefetch. */
+    private int seqReadsBeforePrefetch;
+
+    /** Flag that controls whether file writes should be colocated on data node. */
+    private boolean colocateFileWrites;
+
+    /** Prefer local writes. */
+    private boolean preferLocFileWrites;
+
+    /**
+     * @param name URI for file system.
+     * @param cfg Configuration.
+     * @throws URISyntaxException if name has invalid syntax.
+     * @throws IOException If initialization failed.
+     */
+    public IgniteHadoopFileSystem(URI name, Configuration cfg) throws URISyntaxException, IOException {
+        super(HadoopIgfsEndpoint.normalize(name), IGFS_SCHEME, false, -1);
+
+        uri = name;
+
+        try {
+            initialize(name, cfg);
+        }
+        catch (IOException e) {
+            // Close client if exception occurred.
+            if (rmtClient != null)
+                rmtClient.close(false);
+
+            throw e;
+        }
+
+        workingDir = new IgfsPath("/user/" + cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkPath(Path path) {
+        URI uri = path.toUri();
+
+        if (uri.isAbsolute()) {
+            if (!F.eq(uri.getScheme(), IGFS_SCHEME))
+                throw new InvalidPathException("Wrong path scheme [expected=" + IGFS_SCHEME + ", actual=" +
+                    uri.getAuthority() + ']');
+
+            if (!F.eq(uri.getAuthority(), uriAuthority))
+                throw new InvalidPathException("Wrong path authority [expected=" + uriAuthority + ", actual=" +
+                    uri.getAuthority() + ']');
+        }
+    }
+
+    /**
+     * Public setter that can be used by direct users of FS or Visor.
+     *
+     * @param colocateFileWrites Whether all ongoing file writes should be colocated.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void colocateFileWrites(boolean colocateFileWrites) {
+        this.colocateFileWrites = colocateFileWrites;
+    }
+
+    /**
+     * Enter busy state.
+     *
+     * @throws IOException If file system is stopped.
+     */
+    private void enterBusy() throws IOException {
+        if (closeGuard.get())
+            throw new IOException("File system is stopped.");
+    }
+
+    /**
+     * Leave busy state.
+     */
+    private void leaveBusy() {
+        // No-op.
+    }
+
+    /**
+     * @param name URI passed to constructor.
+     * @param cfg Configuration passed to constructor.
+     * @throws IOException If initialization failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private void initialize(URI name, Configuration cfg) throws IOException {
+        enterBusy();
+
+        try {
+            if (rmtClient != null)
+                throw new IOException("File system is already initialized: " + rmtClient);
+
+            A.notNull(name, "name");
+            A.notNull(cfg, "cfg");
+
+            if (!IGFS_SCHEME.equals(name.getScheme()))
+                throw new IOException("Illegal file system URI [expected=" + IGFS_SCHEME +
+                    "://[name]/[optional_path], actual=" + name + ']');
+
+            uriAuthority = name.getAuthority();
+
+            // Override sequential reads before prefetch if needed.
+            seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
+
+            if (seqReadsBeforePrefetch > 0)
+                seqReadsBeforePrefetchOverride = true;
+
+            // In Ignite replication factor is controlled by data cache affinity.
+            // We use replication factor to force the whole file to be stored on local node.
+            dfltReplication = (short)cfg.getInt("dfs.replication", 3);
+
+            // Get file colocation control flag.
+            colocateFileWrites = parameter(cfg, PARAM_IGFS_COLOCATED_WRITES, uriAuthority, false);
+            preferLocFileWrites = cfg.getBoolean(PARAM_IGFS_PREFER_LOCAL_WRITES, false);
+
+            // Get log directory.
+            String logDirCfg = parameter(cfg, PARAM_IGFS_LOG_DIR, uriAuthority, DFLT_IGFS_LOG_DIR);
+
+            File logDirFile = U.resolveIgnitePath(logDirCfg);
+
+            String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
+
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+
+            // Handshake.
+            IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
+
+            grpBlockSize = handshake.blockSize();
+
+            IgfsPaths paths = handshake.secondaryPaths();
+
+            Boolean logEnabled = parameter(cfg, PARAM_IGFS_LOG_ENABLED, uriAuthority, false);
+
+            if (handshake.sampling() != null ? handshake.sampling() : logEnabled) {
+                // Initiate client logger.
+                if (logDir == null)
+                    throw new IOException("Failed to resolve log directory: " + logDirCfg);
+
+                Integer batchSize = parameter(cfg, PARAM_IGFS_LOG_BATCH_SIZE, uriAuthority, DFLT_IGFS_LOG_BATCH_SIZE);
+
+                clientLog = IgfsLogger.logger(uriAuthority, handshake.igfsName(), logDir, batchSize);
+            }
+            else
+                clientLog = IgfsLogger.disabledLogger();
+
+            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
+
+            boolean initSecondary = paths.defaultMode() == PROXY;
+
+            if (!initSecondary && paths.pathModes() != null) {
+                for (T2<IgfsPath, IgfsMode> pathMode : paths.pathModes()) {
+                    IgfsMode mode = pathMode.getValue();
+
+                    if (mode == PROXY) {
+                        initSecondary = true;
+
+                        break;
+                    }
+                }
+            }
+
+            if (initSecondary) {
+                Map<String, String> props = paths.properties();
+
+                String secUri = props.get(SECONDARY_FS_URI);
+                String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
+
+                try {
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createAbstractFileSystem();
+                    secondaryUri = secProvider.uri();
+                }
+                catch (IOException e) {
+                    throw new IOException("Failed to connect to the secondary file system: " + secUri, e);
+                }
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        if (closeGuard.compareAndSet(false, true)) {
+            if (rmtClient == null)
+                return;
+
+            rmtClient.close(false);
+
+            if (clientLog.isLogEnabled())
+                clientLog.close();
+
+            // Reset initialized resources.
+            rmtClient = null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public URI getUri() {
+        return uri;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getUriDefaultPort() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FsServerDefaults getServerDefaults() throws IOException {
+        return new FsServerDefaults(grpBlockSize, (int)grpBlockSize, (int)grpBlockSize, dfltReplication, 64 * 1024,
+            false, 0, DataChecksum.Type.NULL);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setReplication(Path f, short replication) throws IOException {
+        return mode(f) == PROXY && secondaryFs.setReplication(f, replication);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setTimes(Path f, long mtime, long atime) throws IOException {
+        if (mode(f) == PROXY)
+            secondaryFs.setTimes(f, mtime, atime);
+        else {
+            if (mtime == -1 && atime == -1)
+                return;
+
+            rmtClient.setTimes(convert(f), atime, mtime);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FsStatus getFsStatus() throws IOException {
+        IgfsStatus status = rmtClient.fsStatus();
+
+        return new FsStatus(status.spaceTotal(), status.spaceUsed(), status.spaceTotal() - status.spaceUsed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setPermission(Path p, FsPermission perm) throws IOException {
+        enterBusy();
+
+        try {
+            A.notNull(p, "p");
+
+            if (mode(p) == PROXY)
+                secondaryFs.setPermission(toSecondary(p), perm);
+            else {
+                if (rmtClient.update(convert(p), permission(perm)) == null)
+                    throw new IOException("Failed to set file permission (file not found?)" +
+                        " [path=" + p + ", perm=" + perm + ']');
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setOwner(Path p, String usr, String grp) throws IOException {
+        A.notNull(p, "p");
+        A.notNull(usr, "username");
+        A.notNull(grp, "grpName");
+
+        enterBusy();
+
+        try {
+            if (mode(p) == PROXY)
+                secondaryFs.setOwner(toSecondary(p), usr, grp);
+            else if (rmtClient.update(convert(p), F.asMap(PROP_USER_NAME, usr, PROP_GROUP_NAME, grp)) == null)
+                throw new IOException("Failed to set file permission (file not found?)" +
+                    " [path=" + p + ", username=" + usr + ", grpName=" + grp + ']');
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FSDataInputStream open(Path f, int bufSize) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = modeRslvr.resolveMode(path);
+
+            if (mode == PROXY) {
+                FSDataInputStream is = secondaryFs.open(toSecondary(f), bufSize);
+
+                if (clientLog.isLogEnabled()) {
+                    // At this point we do not know file size, so we perform additional request to remote FS to get it.
+                    FileStatus status = secondaryFs.getFileStatus(toSecondary(f));
+
+                    long size = status != null ? status.getLen() : -1;
+
+                    long logId = IgfsLogger.nextId();
+
+                    clientLog.logOpen(logId, path, PROXY, bufSize, size);
+
+                    return new FSDataInputStream(new HadoopIgfsProxyInputStream(is, clientLog, logId));
+                }
+                else
+                    return is;
+            }
+            else {
+                HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
+                    rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
+
+                long logId = -1;
+
+                if (clientLog.isLogEnabled()) {
+                    logId = IgfsLogger.nextId();
+
+                    clientLog.logOpen(logId, path, mode, bufSize, stream.length());
+                }
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
+                        ", bufSize=" + bufSize + ']');
+
+                HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
+                    bufSize, LOG, clientLog, logId);
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
+
+                return new FSDataInputStream(igfsIn);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public FSDataOutputStream createInternal(
+        Path f,
+        EnumSet<CreateFlag> flag,
+        FsPermission perm,
+        int bufSize,
+        short replication,
+        long blockSize,
+        Progressable progress,
+        Options.ChecksumOpt checksumOpt,
+        boolean createParent
+    ) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
+        boolean append = flag.contains(CreateFlag.APPEND);
+        boolean create = flag.contains(CreateFlag.CREATE);
+
+        OutputStream out = null;
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = modeRslvr.resolveMode(path);
+
+            if (LOG.isDebugEnabled())
+                LOG.debug("Opening output stream in create [thread=" + Thread.currentThread().getName() + "path=" +
+                    path + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ']');
+
+            if (mode == PROXY) {
+                FSDataOutputStream os = secondaryFs.createInternal(toSecondary(f), flag, perm, bufSize,
+                    replication, blockSize, progress, checksumOpt, createParent);
+
+                if (clientLog.isLogEnabled()) {
+                    long logId = IgfsLogger.nextId();
+
+                    if (append)
+                        clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
+                    else
+                        clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
+
+                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
+                }
+                else
+                    return os;
+            }
+            else {
+                Map<String, String> permMap = F.asMap(PROP_PERMISSION, toString(perm),
+                    PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites));
+
+                // Create stream and close it in the 'finally' section if any sequential operation failed.
+                HadoopIgfsStreamDelegate stream;
+
+                long logId = -1;
+
+                if (append) {
+                    stream = rmtClient.append(path, create, permMap);
+
+                    if (clientLog.isLogEnabled()) {
+                        logId = IgfsLogger.nextId();
+
+                        clientLog.logAppend(logId, path, mode, bufSize);
+                    }
+
+                    if (LOG.isDebugEnabled())
+                        LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
+                }
+                else {
+                    stream = rmtClient.create(path, overwrite, colocateFileWrites, replication, blockSize,
+                        permMap);
+
+                    if (clientLog.isLogEnabled()) {
+                        logId = IgfsLogger.nextId();
+
+                        clientLog.logCreate(logId, path, mode, overwrite, bufSize, replication, blockSize);
+                    }
+
+                    if (LOG.isDebugEnabled())
+                        LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
+                }
+
+                assert stream != null;
+
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG,
+                    clientLog, logId);
+
+                bufSize = Math.max(64 * 1024, bufSize);
+
+                out = new BufferedOutputStream(igfsOut, bufSize);
+
+                FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
+
+                // Mark stream created successfully.
+                out = null;
+
+                return res;
+            }
+        }
+        finally {
+            // Close if failed during stream creation.
+            if (out != null)
+                U.closeQuiet(out);
+
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSymlinks() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void renameInternal(Path src, Path dst) throws IOException {
+        A.notNull(src, "src");
+        A.notNull(dst, "dst");
+
+        enterBusy();
+
+        try {
+            IgfsPath srcPath = convert(src);
+            IgfsPath dstPath = convert(dst);
+            Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(srcPath);
+
+            if (childrenModes.contains(PROXY)) {
+                if (clientLog.isLogEnabled())
+                    clientLog.logRename(srcPath, PROXY, dstPath);
+
+                secondaryFs.renameInternal(toSecondary(src), toSecondary(dst));
+            }
+
+            rmtClient.rename(srcPath, dstPath);
+
+            if (clientLog.isLogEnabled())
+                clientLog.logRename(srcPath, modeRslvr.resolveMode(srcPath), dstPath);
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(Path f, boolean recursive) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = modeRslvr.resolveMode(path);
+            Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
+
+            if (childrenModes.contains(PROXY)) {
+                if (clientLog.isLogEnabled())
+                    clientLog.logDelete(path, PROXY, recursive);
+
+                return secondaryFs.delete(toSecondary(f), recursive);
+            }
+
+            boolean res = rmtClient.delete(path, recursive);
+
+            if (clientLog.isLogEnabled())
+                clientLog.logDelete(path, mode, recursive);
+
+            return res;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setVerifyChecksum(boolean verifyChecksum) throws IOException {
+        // Checksum has effect for secondary FS only.
+        if (secondaryFs != null)
+            secondaryFs.setVerifyChecksum(verifyChecksum);
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileChecksum getFileChecksum(Path f) throws IOException {
+        if (mode(f) == PROXY)
+            return secondaryFs.getFileChecksum(f);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus[] listStatus(Path f) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = modeRslvr.resolveMode(path);
+
+            if (mode == PROXY) {
+                FileStatus[] arr = secondaryFs.listStatus(toSecondary(f));
+
+                if (arr == null)
+                    throw new FileNotFoundException("File " + f + " does not exist.");
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = toPrimary(arr[i]);
+
+                if (clientLog.isLogEnabled()) {
+                    String[] fileArr = new String[arr.length];
+
+                    for (int i = 0; i < arr.length; i++)
+                        fileArr[i] = arr[i].getPath().toString();
+
+                    clientLog.logListDirectory(path, PROXY, fileArr);
+                }
+
+                return arr;
+            }
+            else {
+                Collection<IgfsFile> list = rmtClient.listFiles(path);
+
+                if (list == null)
+                    throw new FileNotFoundException("File " + f + " does not exist.");
+
+                List<IgfsFile> files = new ArrayList<>(list);
+
+                FileStatus[] arr = new FileStatus[files.size()];
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = convert(files.get(i));
+
+                if (clientLog.isLogEnabled()) {
+                    String[] fileArr = new String[arr.length];
+
+                    for (int i = 0; i < arr.length; i++)
+                        fileArr[i] = arr[i].getPath().toString();
+
+                    clientLog.logListDirectory(path, mode, fileArr);
+                }
+
+                return arr;
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdir(Path f, FsPermission perm, boolean createParent) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            IgfsPath path = convert(f);
+            IgfsMode mode = modeRslvr.resolveMode(path);
+
+            if (mode == PROXY) {
+                if (clientLog.isLogEnabled())
+                    clientLog.logMakeDirectory(path, PROXY);
+
+                secondaryFs.mkdir(toSecondary(f), perm, createParent);
+            }
+            else {
+                rmtClient.mkdirs(path, permission(perm));
+
+                if (clientLog.isLogEnabled())
+                    clientLog.logMakeDirectory(path, mode);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus getFileStatus(Path f) throws IOException {
+        A.notNull(f, "f");
+
+        enterBusy();
+
+        try {
+            if (mode(f) == PROXY)
+                return toPrimary(secondaryFs.getFileStatus(toSecondary(f)));
+            else {
+                IgfsFile info = rmtClient.info(convert(f));
+
+                if (info == null)
+                    throw new FileNotFoundException("File not found: " + f);
+
+                return convert(info);
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public BlockLocation[] getFileBlockLocations(Path path, long start, long len) throws IOException {
+        A.notNull(path, "path");
+
+        IgfsPath igfsPath = convert(path);
+
+        enterBusy();
+
+        try {
+            if (modeRslvr.resolveMode(igfsPath) == PROXY)
+                return secondaryFs.getFileBlockLocations(path, start, len);
+            else {
+                long now = System.currentTimeMillis();
+
+                List<IgfsBlockLocation> affinity = new ArrayList<>(
+                    rmtClient.affinity(igfsPath, start, len));
+
+                BlockLocation[] arr = new BlockLocation[affinity.size()];
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = convert(affinity.get(i));
+
+                if (LOG.isDebugEnabled())
+                    LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
+                        (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
+
+                return arr;
+            }
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /**
+     * Resolve path mode.
+     *
+     * @param path HDFS path.
+     * @return Path mode.
+     */
+    public IgfsMode mode(Path path) {
+        return modeRslvr.resolveMode(convert(path));
+    }
+
+    /**
+     * Convert the given path to path acceptable by the primary file system.
+     *
+     * @param path Path.
+     * @return Primary file system path.
+     */
+    private Path toPrimary(Path path) {
+        return convertPath(path, getUri());
+    }
+
+    /**
+     * Convert the given path to path acceptable by the secondary file system.
+     *
+     * @param path Path.
+     * @return Secondary file system path.
+     */
+    private Path toSecondary(Path path) {
+        assert secondaryFs != null;
+        assert secondaryUri != null;
+
+        return convertPath(path, secondaryUri);
+    }
+
+    /**
+     * Convert path using the given new URI.
+     *
+     * @param path Old path.
+     * @param newUri New URI.
+     * @return New path.
+     */
+    private Path convertPath(Path path, URI newUri) {
+        assert newUri != null;
+
+        if (path != null) {
+            URI pathUri = path.toUri();
+
+            try {
+                return new Path(new URI(pathUri.getScheme() != null ? newUri.getScheme() : null,
+                    pathUri.getAuthority() != null ? newUri.getAuthority() : null, pathUri.getPath(), null, null));
+            }
+            catch (URISyntaxException e) {
+                throw new IgniteException("Failed to construct secondary file system path from the primary file " +
+                    "system path: " + path, e);
+            }
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Convert a file status obtained from the secondary file system to a status of the primary file system.
+     *
+     * @param status Secondary file system status.
+     * @return Primary file system status.
+     */
+    private FileStatus toPrimary(FileStatus status) {
+        return status != null ? new FileStatus(status.getLen(), status.isDirectory(), status.getReplication(),
+            status.getBlockSize(), status.getModificationTime(), status.getAccessTime(), status.getPermission(),
+            status.getOwner(), status.getGroup(), toPrimary(status.getPath())) : null;
+    }
+
+    /**
+     * Convert IGFS path into Hadoop path.
+     *
+     * @param path IGFS path.
+     * @return Hadoop path.
+     */
+    private Path convert(IgfsPath path) {
+        return new Path(IGFS_SCHEME, uriAuthority, path.toString());
+    }
+
+    /**
+     * Convert Hadoop path into IGFS path.
+     *
+     * @param path Hadoop path.
+     * @return IGFS path.
+     */
+    @Nullable private IgfsPath convert(Path path) {
+        if (path == null)
+            return null;
+
+        return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
+            new IgfsPath(workingDir, path.toUri().getPath());
+    }
+
+    /**
+     * Convert IGFS affinity block location into Hadoop affinity block location.
+     *
+     * @param block IGFS affinity block location.
+     * @return Hadoop affinity block location.
+     */
+    private BlockLocation convert(IgfsBlockLocation block) {
+        Collection<String> names = block.names();
+        Collection<String> hosts = block.hosts();
+
+        return new BlockLocation(
+            names.toArray(new String[names.size()]) /* hostname:portNumber of data nodes */,
+            hosts.toArray(new String[hosts.size()]) /* hostnames of data nodes */,
+            block.start(), block.length()
+        ) {
+            @Override public String toString() {
+                try {
+                    return "BlockLocation [offset=" + getOffset() + ", length=" + getLength() +
+                        ", hosts=" + Arrays.asList(getHosts()) + ", names=" + Arrays.asList(getNames()) + ']';
+                }
+                catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Convert IGFS file information into Hadoop file status.
+     *
+     * @param file IGFS file information.
+     * @return Hadoop file status.
+     */
+    private FileStatus convert(IgfsFile file) {
+        return new FileStatus(
+            file.length(),
+            file.isDirectory(),
+            dfltReplication,
+            file.groupBlockSize(),
+            file.modificationTime(),
+            file.accessTime(),
+            permission(file),
+            file.property(PROP_USER_NAME, DFLT_USER_NAME),
+            file.property(PROP_GROUP_NAME, "users"),
+            convert(file.path())) {
+            @Override public String toString() {
+                return "FileStatus [path=" + getPath() + ", isDir=" + isDirectory() + ", len=" + getLen() + "]";
+            }
+        };
+    }
+
+    /**
+     * Convert Hadoop permission into IGFS file attribute.
+     *
+     * @param perm Hadoop permission.
+     * @return IGFS attributes.
+     */
+    private Map<String, String> permission(FsPermission perm) {
+        if (perm == null)
+            perm = FsPermission.getDefault();
+
+        return F.asMap(PROP_PERMISSION, toString(perm));
+    }
+
+    /**
+     * @param perm Permission.
+     * @return String.
+     */
+    private static String toString(FsPermission perm) {
+        return String.format("%04o", perm.toShort());
+    }
+
+    /**
+     * Convert IGFS file attributes into Hadoop permission.
+     *
+     * @param file File info.
+     * @return Hadoop permission.
+     */
+    private FsPermission permission(IgfsFile file) {
+        String perm = file.property(PROP_PERMISSION, null);
+
+        if (perm == null)
+            return FsPermission.getDefault();
+
+        try {
+            return new FsPermission((short)Integer.parseInt(perm, 8));
+        }
+        catch (NumberFormatException ignore) {
+            return FsPermission.getDefault();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteHadoopFileSystem.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/package.html b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/package.html
new file mode 100644
index 0000000..8adac96
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/package.html
@@ -0,0 +1,24 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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 Ignite Hadoop 2.x <code>FileSystem</code> implementation.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
new file mode 100644
index 0000000..83053ce
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.mapreduce;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.protocol.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.client.*;
+import org.apache.ignite.internal.client.marshaller.optimized.*;
+import org.apache.ignite.internal.processors.hadoop.proto.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.client.GridClientProtocol.*;
+import static org.apache.ignite.internal.processors.hadoop.proto.HadoopClientProtocol.*;
+
+
+/**
+ * Ignite Hadoop client protocol provider.
+ */
+public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider {
+    /** Clients. */
+    private static final ConcurrentHashMap<String, IgniteInternalFuture<GridClient>> cliMap = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public ClientProtocol create(Configuration conf) throws IOException {
+        if (FRAMEWORK_NAME.equals(conf.get(MRConfig.FRAMEWORK_NAME))) {
+            String addr = conf.get(MRConfig.MASTER_ADDRESS);
+
+            if (F.isEmpty(addr))
+                throw new IOException("Failed to create client protocol because server address is not specified (is " +
+                    MRConfig.MASTER_ADDRESS + " property set?).");
+
+            if (F.eq(addr, "local"))
+                throw new IOException("Local execution mode is not supported, please point " +
+                    MRConfig.MASTER_ADDRESS + " to real Ignite node.");
+
+            return createProtocol(addr, conf);
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClientProtocol create(InetSocketAddress addr, Configuration conf) throws IOException {
+        if (FRAMEWORK_NAME.equals(conf.get(MRConfig.FRAMEWORK_NAME)))
+            return createProtocol(addr.getHostString() + ":" + addr.getPort(), conf);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(ClientProtocol cliProto) throws IOException {
+        // No-op.
+    }
+
+    /**
+     * Internal protocol creation routine.
+     *
+     * @param addr Address.
+     * @param conf Configuration.
+     * @return Client protocol.
+     * @throws IOException If failed.
+     */
+    private static ClientProtocol createProtocol(String addr, Configuration conf) throws IOException {
+        return new HadoopClientProtocol(conf, client(addr));
+    }
+
+    /**
+     * Create client.
+     *
+     * @param addr Endpoint address.
+     * @return Client.
+     * @throws IOException If failed.
+     */
+    private static GridClient client(String addr) throws IOException {
+        try {
+            IgniteInternalFuture<GridClient> fut = cliMap.get(addr);
+
+            if (fut == null) {
+                GridFutureAdapter<GridClient> fut0 = new GridFutureAdapter<>();
+
+                IgniteInternalFuture<GridClient> oldFut = cliMap.putIfAbsent(addr, fut0);
+
+                if (oldFut != null)
+                    return oldFut.get();
+                else {
+                    GridClientConfiguration cliCfg = new GridClientConfiguration();
+
+                    cliCfg.setProtocol(TCP);
+                    cliCfg.setServers(Collections.singletonList(addr));
+                    cliCfg.setMarshaller(new GridClientOptimizedMarshaller());
+                    cliCfg.setDaemon(true);
+
+                    try {
+                        GridClient cli = GridClientFactory.start(cliCfg);
+
+                        fut0.onDone(cli);
+
+                        return cli;
+                    }
+                    catch (GridClientException e) {
+                        fut0.onDone(e);
+
+                        throw new IOException("Failed to establish connection with Ignite node: " + addr, e);
+                    }
+                }
+            }
+            else
+                return fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException("Failed to establish connection with Ignite node: " + addr, e);
+        }
+    }
+}


[26/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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 c734acd..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);
-        GridHadoopDefaultJobInfo 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/6423cf02/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/6423cf02/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/6423cf02/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
new file mode 100644
index 0000000..b454760
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
@@ -0,0 +1,333 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.conf.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.protocol.*;
+import org.apache.hadoop.mapreduce.security.token.delegation.*;
+import org.apache.hadoop.mapreduce.v2.*;
+import org.apache.hadoop.mapreduce.v2.jobhistory.*;
+import org.apache.hadoop.security.*;
+import org.apache.hadoop.security.authorize.*;
+import org.apache.hadoop.security.token.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.client.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Hadoop client protocol.
+ */
+public class HadoopClientProtocol implements ClientProtocol {
+    /** Ignite framework name property. */
+    public static final String FRAMEWORK_NAME = "ignite";
+
+    /** Protocol version. */
+    private static final long PROTO_VER = 1L;
+
+    /** Default Ignite system directory. */
+    private static final String SYS_DIR = ".ignite/system";
+
+    /** Configuration. */
+    private final Configuration conf;
+
+    /** Ignite client. */
+    private volatile GridClient cli;
+
+    /** Last received version. */
+    private long lastVer = -1;
+
+    /** Last received status. */
+    private HadoopJobStatus lastStatus;
+
+    /**
+     * Constructor.
+     *
+     * @param conf Configuration.
+     * @param cli Ignite client.
+     */
+    public HadoopClientProtocol(Configuration conf, GridClient cli) {
+        assert cli != null;
+
+        this.conf = conf;
+        this.cli = cli;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JobID getNewJobID() throws IOException, InterruptedException {
+        try {
+            conf.setLong(REQ_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
+
+            HadoopJobId jobID = cli.compute().execute(HadoopProtocolNextTaskIdTask.class.getName(), null);
+
+            conf.setLong(RESPONSE_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
+
+            return new JobID(jobID.globalId().toString(), jobID.localId());
+        }
+        catch (GridClientException e) {
+            throw new IOException("Failed to get new job ID.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) throws IOException,
+        InterruptedException {
+        try {
+            conf.setLong(JOB_SUBMISSION_START_TS_PROPERTY, U.currentTimeMillis());
+
+            HadoopJobStatus 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);
+
+            return processStatus(status);
+        }
+        catch (GridClientException | IgniteCheckedException e) {
+            throw new IOException("Failed to submit job.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterMetrics getClusterMetrics() throws IOException, InterruptedException {
+        return new ClusterMetrics(0, 0, 0, 0, 0, 0, 1000, 1000, 1, 100, 0, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Cluster.JobTrackerStatus getJobTrackerStatus() throws IOException, InterruptedException {
+        return Cluster.JobTrackerStatus.RUNNING;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getTaskTrackerExpiryInterval() throws IOException, InterruptedException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public AccessControlList getQueueAdmins(String queueName) throws IOException {
+        return new AccessControlList("*");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void killJob(JobID jobId) throws IOException, InterruptedException {
+        try {
+            cli.compute().execute(HadoopProtocolKillJobTask.class.getName(),
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
+        }
+        catch (GridClientException e) {
+            throw new IOException("Failed to kill job: " + jobId, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setJobPriority(JobID jobid, String priority) throws IOException, InterruptedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException,
+        InterruptedException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public JobStatus getJobStatus(JobID jobId) throws IOException, InterruptedException {
+        try {
+            Long delay = conf.getLong(HadoopJobProperty.JOB_STATUS_POLL_DELAY.propertyName(), -1);
+
+            HadoopProtocolTaskArguments args = delay >= 0 ?
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), delay) :
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId());
+
+            HadoopJobStatus 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);
+
+            return processStatus(status);
+        }
+        catch (GridClientException e) {
+            throw new IOException("Failed to get job status: " + jobId, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counters getJobCounters(JobID jobId) throws IOException, InterruptedException {
+        try {
+            final HadoopCounters 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);
+
+            return new HadoopMapReduceCounters(counters);
+        }
+        catch (GridClientException e) {
+            throw new IOException("Failed to get job counters: " + jobId, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public TaskReport[] getTaskReports(JobID jobid, TaskType type) throws IOException, InterruptedException {
+        return new TaskReport[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getFilesystemName() throws IOException, InterruptedException {
+        return FileSystem.get(conf).getUri().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public JobStatus[] getAllJobs() throws IOException, InterruptedException {
+        return new JobStatus[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public TaskCompletionEvent[] getTaskCompletionEvents(JobID jobid, int fromEventId, int maxEvents)
+        throws IOException, InterruptedException {
+        return new TaskCompletionEvent[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public String[] getTaskDiagnostics(TaskAttemptID taskId) throws IOException, InterruptedException {
+        return new String[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public TaskTrackerInfo[] getActiveTrackers() throws IOException, InterruptedException {
+        return new TaskTrackerInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public TaskTrackerInfo[] getBlacklistedTrackers() throws IOException, InterruptedException {
+        return new TaskTrackerInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSystemDir() throws IOException, InterruptedException {
+        Path sysDir = new Path(SYS_DIR);
+
+        return sysDir.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getStagingAreaDir() throws IOException, InterruptedException {
+        String usr = UserGroupInformation.getCurrentUser().getShortUserName();
+
+        return HadoopUtils.stagingAreaDir(conf, usr).toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getJobHistoryDir() throws IOException, InterruptedException {
+        return JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueueInfo[] getQueues() throws IOException, InterruptedException {
+        return new QueueInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueueInfo getQueue(String queueName) throws IOException, InterruptedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException, InterruptedException {
+        return new QueueAclsInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueueInfo[] getRootQueues() throws IOException, InterruptedException {
+        return new QueueInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueueInfo[] getChildQueues(String queueName) throws IOException, InterruptedException {
+        return new QueueInfo[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) throws IOException,
+        InterruptedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long renewDelegationToken(Token<DelegationTokenIdentifier> token) throws IOException,
+        InterruptedException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelDelegationToken(Token<DelegationTokenIdentifier> token) throws IOException,
+        InterruptedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public LogParams getLogFileParams(JobID jobID, TaskAttemptID taskAttemptID) throws IOException,
+        InterruptedException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+        return PROTO_VER;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, int clientMethodsHash)
+        throws IOException {
+        return ProtocolSignature.getProtocolSignature(this, protocol, clientVersion, clientMethodsHash);
+    }
+
+    /**
+     * Process received status update.
+     *
+     * @param status Ignite status.
+     * @return Hadoop status.
+     */
+    private JobStatus processStatus(HadoopJobStatus status) {
+        // IMPORTANT! This method will only work in single-threaded environment. It is valid at the moment because
+        // IgniteHadoopClientProtocolProvider creates new instance of this class for every new job and Job class
+        // serializes invocations of submitJob() and getJobStatus() methods. However, if any of these conditions will
+        // change in future and either protocol will serve statuses for several jobs or status update will not be
+        // serialized anymore, then we have to fallback to concurrent approach (e.g. using ConcurrentHashMap).
+        // (vozerov)
+        if (lastVer < status.version()) {
+            lastVer = status.version();
+
+            lastStatus = status;
+        }
+        else
+            assert lastStatus != null;
+
+        return HadoopUtils.status(lastStatus, conf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..ebdda9f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobCountersTask.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 org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+
+import java.util.*;
+
+/**
+ * Task to get job counters.
+ */
+public class HadoopProtocolJobCountersTask extends HadoopProtocolTaskAdapter<HadoopCounters> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public HadoopCounters run(ComputeJobContext jobCtx, Hadoop 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 HadoopJobId(nodeId, id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..1734562
--- /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<HadoopJobStatus> {
+    /** */
+    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 HadoopJobStatus run(final ComputeJobContext jobCtx, Hadoop 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;
+
+        HadoopJobId jobId = new HadoopJobId(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/6423cf02/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..d173612
--- /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, Hadoop hadoop,
+        HadoopProtocolTaskArguments args) throws IgniteCheckedException {
+        UUID nodeId = UUID.fromString(args.<String>get(0));
+        Integer id = args.get(1);
+
+        assert nodeId != null;
+        assert id != null;
+
+        HadoopJobId jobId = new HadoopJobId(nodeId, id);
+
+        return hadoop.kill(jobId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..2782530
--- /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<HadoopJobId> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public HadoopJobId run(ComputeJobContext jobCtx, Hadoop hadoop,
+        HadoopProtocolTaskArguments args) {
+        return hadoop.nextJobId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..f65d9bb
--- /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.HadoopJobPhase.*;
+
+/**
+ * Submit job task.
+ */
+public class HadoopProtocolSubmitJobTask extends HadoopProtocolTaskAdapter<HadoopJobStatus> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public HadoopJobStatus run(ComputeJobContext jobCtx, Hadoop 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;
+
+        HadoopJobId jobId = new HadoopJobId(nodeId, id);
+
+        hadoop.submit(jobId, info);
+
+        HadoopJobStatus res = hadoop.status(jobId);
+
+        if (res == null) // Submission failed.
+            res = new HadoopJobStatus(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/6423cf02/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..f763ccc
--- /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, Hadoop hadoop, HadoopProtocolTaskArguments args)
+        throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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;
-    }
-}


[03/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java
new file mode 100644
index 0000000..a2f2ac3
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.google.common.collect.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.Map.*;
+
+import static com.google.common.collect.Maps.*;
+import static com.google.common.collect.MinMaxPriorityQueue.*;
+import static java.util.Collections.*;
+
+/**
+ * Hadoop-based 10 popular words example: all files in a given directory are tokenized and for each word longer than
+ * 3 characters the number of occurrences ins calculated. Finally, 10 words with the highest occurrence count are
+ * output.
+ *
+ * NOTE: in order to run this example on Windows please ensure that cygwin is installed and available in the system
+ * path.
+ */
+public class HadoopPopularWordsTest {
+    /** Ignite home. */
+    private static final String IGNITE_HOME = U.getIgniteHome();
+
+    /** The path to the input directory. ALl files in that directory will be processed. */
+    private static final Path BOOKS_LOCAL_DIR =
+        new Path("file:" + IGNITE_HOME, "modules/tests/java/org/apache/ignite/grid/hadoop/books");
+
+    /** The path to the output directory. THe result file will be written to this location. */
+    private static final Path RESULT_LOCAL_DIR =
+        new Path("file:" + IGNITE_HOME, "modules/tests/java/org/apache/ignite/grid/hadoop/output");
+
+    /** Popular books source dir in DFS. */
+    private static final Path BOOKS_DFS_DIR = new Path("tmp/word-count-example/in");
+
+    /** Popular books source dir in DFS. */
+    private static final Path RESULT_DFS_DIR = new Path("tmp/word-count-example/out");
+
+    /** Path to the distributed file system configuration. */
+    private static final String DFS_CFG = "examples/config/filesystem/core-site.xml";
+
+    /** Top N words to select **/
+    private static final int POPULAR_WORDS_CNT = 10;
+
+    /**
+     * For each token in the input string the mapper emits a {word, 1} pair.
+     */
+    private static class TokenizingMapper extends Mapper<LongWritable, Text, Text, IntWritable> {
+        /** Constant value. */
+        private static final IntWritable ONE = new IntWritable(1);
+
+        /** The word converted into the Text. */
+        private Text word = new Text();
+
+        /**
+         * Emits a entry where the key is the word and the value is always 1.
+         *
+         * @param key the current position in the input file (not used here)
+         * @param val the text string
+         * @param ctx mapper context
+         * @throws IOException
+         * @throws InterruptedException
+         */
+        @Override protected void map(LongWritable key, Text val, Context ctx)
+            throws IOException, InterruptedException {
+            // Get the mapped object.
+            final String line = val.toString();
+
+            // Splits the given string to words.
+            final String[] words = line.split("[^a-zA-Z0-9]");
+
+            for (final String w : words) {
+                // Only emit counts for longer words.
+                if (w.length() <= 3)
+                    continue;
+
+                word.set(w);
+
+                // Write the word into the context with the initial count equals 1.
+                ctx.write(word, ONE);
+            }
+        }
+    }
+
+    /**
+     * The reducer uses a priority queue to rank the words based on its number of occurrences.
+     */
+    private static class TopNWordsReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
+        private MinMaxPriorityQueue<Entry<Integer, String>> q;
+
+        TopNWordsReducer() {
+            q = orderedBy(reverseOrder(new Comparator<Entry<Integer, String>>() {
+                @Override public int compare(Entry<Integer, String> o1, Entry<Integer, String> o2) {
+                    return o1.getKey().compareTo(o2.getKey());
+                }
+            })).expectedSize(POPULAR_WORDS_CNT).maximumSize(POPULAR_WORDS_CNT).create();
+        }
+
+        /**
+         * This method doesn't emit anything, but just keeps track of the top N words.
+         *
+         * @param key The word.
+         * @param vals The words counts.
+         * @param ctx Reducer context.
+         * @throws IOException If failed.
+         * @throws InterruptedException If failed.
+         */
+        @Override public void reduce(Text key, Iterable<IntWritable> vals, Context ctx) throws IOException,
+            InterruptedException {
+            int sum = 0;
+
+            for (IntWritable val : vals)
+                sum += val.get();
+
+            q.add(immutableEntry(sum, key.toString()));
+        }
+
+        /**
+         * This method is called after all the word entries have been processed. It writes the accumulated
+         * statistics to the job output file.
+         *
+         * @param ctx The job context.
+         * @throws IOException If failed.
+         * @throws InterruptedException If failed.
+         */
+        @Override protected void cleanup(Context ctx) throws IOException, InterruptedException {
+            IntWritable i = new IntWritable();
+
+            Text txt = new Text();
+
+            // iterate in desc order
+            while (!q.isEmpty()) {
+                Entry<Integer, String> e = q.removeFirst();
+
+                i.set(e.getKey());
+
+                txt.set(e.getValue());
+
+                ctx.write(txt, i);
+            }
+        }
+    }
+
+    /**
+     * Configures the Hadoop MapReduce job.
+     *
+     * @return Instance of the Hadoop MapRed job.
+     * @throws IOException If failed.
+     */
+    private Job createConfigBasedHadoopJob() throws IOException {
+        Job jobCfg = new Job();
+
+        Configuration cfg = jobCfg.getConfiguration();
+
+        // Use explicit configuration of distributed file system, if provided.
+        if (DFS_CFG != null)
+            cfg.addResource(U.resolveIgniteUrl(DFS_CFG));
+
+        jobCfg.setJobName("HadoopPopularWordExample");
+        jobCfg.setJarByClass(HadoopPopularWordsTest.class);
+        jobCfg.setInputFormatClass(TextInputFormat.class);
+        jobCfg.setOutputKeyClass(Text.class);
+        jobCfg.setOutputValueClass(IntWritable.class);
+        jobCfg.setMapperClass(TokenizingMapper.class);
+        jobCfg.setReducerClass(TopNWordsReducer.class);
+
+        FileInputFormat.setInputPaths(jobCfg, BOOKS_DFS_DIR);
+        FileOutputFormat.setOutputPath(jobCfg, RESULT_DFS_DIR);
+
+        // Local job tracker allows the only task per wave, but text input format
+        // replaces it with the calculated value based on input split size option.
+        if ("local".equals(cfg.get("mapred.job.tracker", "local"))) {
+            // Split job into tasks using 32MB split size.
+            FileInputFormat.setMinInputSplitSize(jobCfg, 32 * 1024 * 1024);
+            FileInputFormat.setMaxInputSplitSize(jobCfg, Long.MAX_VALUE);
+        }
+
+        return jobCfg;
+    }
+
+    /**
+     * Runs the Hadoop job.
+     *
+     * @return {@code True} if succeeded, {@code false} otherwise.
+     * @throws Exception If failed.
+     */
+    private boolean runWordCountConfigBasedHadoopJob() throws Exception {
+        Job job = createConfigBasedHadoopJob();
+
+        // Distributed file system this job will work with.
+        FileSystem fs = FileSystem.get(job.getConfiguration());
+
+        X.println(">>> Using distributed file system: " + fs.getHomeDirectory());
+
+        // Prepare input and output job directories.
+        prepareDirectories(fs);
+
+        long time = System.currentTimeMillis();
+
+        // Run job.
+        boolean res = job.waitForCompletion(true);
+
+        X.println(">>> Job execution time: " + (System.currentTimeMillis() - time) / 1000 + " sec.");
+
+        // Move job results into local file system, so you can view calculated results.
+        publishResults(fs);
+
+        return res;
+    }
+
+    /**
+     * Prepare job's data: cleanup result directories that might have left over
+     * after previous runs, copy input files from the local file system into DFS.
+     *
+     * @param fs Distributed file system to use in job.
+     * @throws IOException If failed.
+     */
+    private void prepareDirectories(FileSystem fs) throws IOException {
+        X.println(">>> Cleaning up DFS result directory: " + RESULT_DFS_DIR);
+
+        fs.delete(RESULT_DFS_DIR, true);
+
+        X.println(">>> Cleaning up DFS input directory: " + BOOKS_DFS_DIR);
+
+        fs.delete(BOOKS_DFS_DIR, true);
+
+        X.println(">>> Copy local files into DFS input directory: " + BOOKS_DFS_DIR);
+
+        fs.copyFromLocalFile(BOOKS_LOCAL_DIR, BOOKS_DFS_DIR);
+    }
+
+    /**
+     * Publish job execution results into local file system, so you can view them.
+     *
+     * @param fs Distributed file sytem used in job.
+     * @throws IOException If failed.
+     */
+    private void publishResults(FileSystem fs) throws IOException {
+        X.println(">>> Cleaning up DFS input directory: " + BOOKS_DFS_DIR);
+
+        fs.delete(BOOKS_DFS_DIR, true);
+
+        X.println(">>> Cleaning up LOCAL result directory: " + RESULT_LOCAL_DIR);
+
+        fs.delete(RESULT_LOCAL_DIR, true);
+
+        X.println(">>> Moving job results into LOCAL result directory: " + RESULT_LOCAL_DIR);
+
+        fs.copyToLocalFile(true, RESULT_DFS_DIR, RESULT_LOCAL_DIR);
+    }
+
+    /**
+     * Executes a modified version of the Hadoop word count example. Here, in addition to counting the number of
+     * occurrences of the word in the source files, the N most popular words are selected.
+     *
+     * @param args None.
+     */
+    public static void main(String[] args) {
+        try {
+            new HadoopPopularWordsTest().runWordCountConfigBasedHadoopJob();
+        }
+        catch (Exception e) {
+            X.println(">>> Failed to run word count example: " + e.getMessage());
+        }
+
+        System.exit(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..5d5bb94
--- /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 {
+        HadoopSerialization 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 {
+        HadoopSerialization 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/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java
new file mode 100644
index 0000000..c73ee9f
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.jdk8.backport.*;
+
+import java.util.concurrent.*;
+
+/**
+ * For tests.
+ */
+public class HadoopSharedMap {
+    /** */
+    private static final ConcurrentMap<String, HadoopSharedMap> maps = new ConcurrentHashMap8<>();
+
+    /** */
+    private final ConcurrentMap<String, Object> map = new ConcurrentHashMap8<>();
+
+    /**
+     * Private.
+     */
+    private HadoopSharedMap() {
+        // No-op.
+    }
+
+    /**
+     * Puts object by key.
+     *
+     * @param key Key.
+     * @param val Value.
+     */
+    public <T> T put(String key, T val) {
+        Object old = map.putIfAbsent(key, val);
+
+        return old == null ? val : (T)old;
+    }
+
+    /**
+     * @param cls Class.
+     * @return Map of static fields.
+     */
+    public static HadoopSharedMap map(Class<?> cls) {
+        HadoopSharedMap m = maps.get(cls.getName());
+
+        if (m != null)
+            return m;
+
+        HadoopSharedMap old = maps.putIfAbsent(cls.getName(), m = new HadoopSharedMap());
+
+        return old == null ? m : old;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java
new file mode 100644
index 0000000..772e77d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.configuration.*;
+
+/**
+ * External test for sorting.
+ */
+public class HadoopSortingExternalTest extends HadoopSortingTest {
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(true);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java
new file mode 100644
index 0000000..3f6594a
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Tests correct sorting.
+ */
+public class HadoopSortingTest extends HadoopAbstractSelfTest {
+    /** */
+    private static final String PATH_INPUT = "/test-in";
+
+    /** */
+    private static final String PATH_OUTPUT = "/test-out";
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /**
+     * @return {@code True} if IGFS is enabled on Hadoop nodes.
+     */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSortSimple() throws Exception {
+        // Generate test data.
+        Job job = Job.getInstance();
+
+        job.setInputFormatClass(InFormat.class);
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        job.setMapperClass(Mapper.class);
+        job.setNumReduceTasks(0);
+
+        setupFileSystems(job.getConfiguration());
+
+        FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_INPUT));
+
+        X.printerrln("Data generation started.");
+
+        grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 1),
+            createJobInfo(job.getConfiguration())).get(180000);
+
+        X.printerrln("Data generation complete.");
+
+        // Run main map-reduce job.
+        job = Job.getInstance();
+
+        setupFileSystems(job.getConfiguration());
+
+        job.getConfiguration().set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, JavaSerialization.class.getName() +
+            "," + WritableSerialization.class.getName());
+
+        FileInputFormat.setInputPaths(job, new Path(igfsScheme() + PATH_INPUT));
+        FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
+
+        job.setSortComparatorClass(JavaSerializationComparator.class);
+
+        job.setMapperClass(MyMapper.class);
+        job.setReducerClass(MyReducer.class);
+
+        job.setNumReduceTasks(2);
+
+        job.setMapOutputKeyClass(UUID.class);
+        job.setMapOutputValueClass(NullWritable.class);
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        X.printerrln("Job started.");
+
+        grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 2),
+            createJobInfo(job.getConfiguration())).get(180000);
+
+        X.printerrln("Job complete.");
+
+        // Check result.
+        Path outDir = new Path(igfsScheme() + PATH_OUTPUT);
+
+        AbstractFileSystem fs = AbstractFileSystem.get(new URI(igfsScheme()), job.getConfiguration());
+
+        for (FileStatus file : fs.listStatus(outDir)) {
+            X.printerrln("__ file: " + file);
+
+            if (file.getLen() == 0)
+                continue;
+
+            FSDataInputStream in = fs.open(file.getPath());
+
+            Scanner sc = new Scanner(in);
+
+            UUID prev = null;
+
+            while(sc.hasNextLine()) {
+                UUID next = UUID.fromString(sc.nextLine());
+
+//                X.printerrln("___ check: " + next);
+
+                if (prev != null)
+                    assertTrue(prev.compareTo(next) < 0);
+
+                prev = next;
+            }
+        }
+    }
+
+    public static class InFormat extends InputFormat<Text, NullWritable> {
+        /** {@inheritDoc} */
+        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException, InterruptedException {
+            List<InputSplit> res = new ArrayList<>();
+
+            FakeSplit split = new FakeSplit(20);
+
+            for (int i = 0; i < 10; i++)
+                res.add(split);
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public RecordReader<Text, NullWritable> createRecordReader(final InputSplit split,
+            TaskAttemptContext ctx) throws IOException, InterruptedException {
+            return new RecordReader<Text, NullWritable>() {
+                /** */
+                int cnt;
+
+                /** */
+                Text txt = new Text();
+
+                @Override public void initialize(InputSplit split, TaskAttemptContext ctx) {
+                    // No-op.
+                }
+
+                @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+                    return ++cnt <= split.getLength();
+                }
+
+                @Override public Text getCurrentKey() {
+                    txt.set(UUID.randomUUID().toString());
+
+//                    X.printerrln("___ read: " + txt);
+
+                    return txt;
+                }
+
+                @Override public NullWritable getCurrentValue() {
+                    return NullWritable.get();
+                }
+
+                @Override public float getProgress() throws IOException, InterruptedException {
+                    return (float)cnt / split.getLength();
+                }
+
+                @Override public void close() {
+                    // No-op.
+                }
+            };
+        }
+    }
+
+    public static class MyMapper extends Mapper<LongWritable, Text, UUID, NullWritable> {
+        /** {@inheritDoc} */
+        @Override protected void map(LongWritable key, Text val, Context ctx) throws IOException, InterruptedException {
+//            X.printerrln("___ map: " + val);
+
+            ctx.write(UUID.fromString(val.toString()), NullWritable.get());
+        }
+    }
+
+    public static class MyReducer extends Reducer<UUID, NullWritable, Text, NullWritable> {
+        /** */
+        private Text text = new Text();
+
+        /** {@inheritDoc} */
+        @Override protected void reduce(UUID key, Iterable<NullWritable> vals, Context ctx)
+            throws IOException, InterruptedException {
+//            X.printerrln("___ rdc: " + key);
+
+            text.set(key.toString());
+
+            ctx.write(text, NullWritable.get());
+        }
+    }
+
+    public static class FakeSplit extends InputSplit implements Writable {
+        /** */
+        private static final String[] HOSTS = {"127.0.0.1"};
+
+        /** */
+        private int len;
+
+        /**
+         * @param len Length.
+         */
+        public FakeSplit(int len) {
+            this.len = len;
+        }
+
+        /**
+         *
+         */
+        public FakeSplit() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getLength() throws IOException, InterruptedException {
+            return len;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String[] getLocations() throws IOException, InterruptedException {
+            return HOSTS;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(DataOutput out) throws IOException {
+            out.writeInt(len);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readFields(DataInput in) throws IOException {
+            len = in.readInt();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..ee490be
--- /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 HadoopAbstractSelfTest {
+    /**
+     * 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/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java
new file mode 100644
index 0000000..1a93223
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.ignite.hadoop.fs.v2.IgniteHadoopFileSystem;
+import org.apache.ignite.internal.util.typedef.*;
+
+/**
+ * Hadoop node startup.
+ */
+public class HadoopStartup {
+    /**
+     * @param args Arguments.
+     */
+    public static void main(String[] args) {
+        G.start("config/hadoop/default-config.xml");
+    }
+
+    /**
+     * @return Configuration for job run.
+     */
+    @SuppressWarnings("UnnecessaryFullyQualifiedName")
+    public static Configuration configuration() {
+        Configuration cfg = new Configuration();
+
+        cfg.set("fs.defaultFS", "igfs://igfs@localhost");
+
+        cfg.set("fs.igfs.impl", org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.class.getName());
+        cfg.set("fs.AbstractFileSystem.igfs.impl", IgniteHadoopFileSystem.class.getName());
+
+        cfg.set("dfs.client.block.write.replace-datanode-on-failure.policy", "NEVER");
+
+        cfg.set("mapreduce.framework.name", "ignite");
+        cfg.set("mapreduce.jobtracker.address", "localhost:11211");
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
new file mode 100644
index 0000000..20c5db2
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
@@ -0,0 +1,551 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Tests map-reduce task execution basics.
+ */
+public class HadoopTaskExecutionSelfTest extends HadoopAbstractSelfTest {
+    /** */
+    private static HadoopSharedMap m = HadoopSharedMap.map(HadoopTaskExecutionSelfTest.class);
+
+    /** Line count. */
+    private static final AtomicInteger totalLineCnt = m.put("totalLineCnt", new AtomicInteger());
+
+    /** Executed tasks. */
+    private static final AtomicInteger executedTasks = m.put("executedTasks", new AtomicInteger());
+
+    /** Cancelled tasks. */
+    private static final AtomicInteger cancelledTasks = m.put("cancelledTasks", new AtomicInteger());
+
+    /** Working directory of each task. */
+    private static final Map<String, String> taskWorkDirs = m.put("taskWorkDirs",
+        new ConcurrentHashMap<String, String>());
+
+    /** Mapper id to fail. */
+    private static final AtomicInteger failMapperId = m.put("failMapperId", new AtomicInteger());
+
+    /** Number of splits of the current input. */
+    private static final AtomicInteger splitsCount = m.put("splitsCount", new AtomicInteger());
+
+    /** Test param. */
+    private static final String MAP_WRITE = "test.map.write";
+
+
+    /** {@inheritDoc} */
+    @Override public FileSystemConfiguration igfsConfiguration() {
+        FileSystemConfiguration cfg = super.igfsConfiguration();
+
+        cfg.setFragmentizerEnabled(false);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        grid(0).fileSystem(igfsName).format();
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setMaxParallelTasks(5);
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapRun() throws Exception {
+        int lineCnt = 10000;
+        String fileName = "/testFile";
+
+        prepareFile(fileName, lineCnt);
+
+        totalLineCnt.set(0);
+        taskWorkDirs.clear();
+
+        Configuration cfg = new Configuration();
+
+        cfg.setStrings("fs.igfs.impl", IgniteHadoopFileSystem.class.getName());
+
+        Job job = Job.getInstance(cfg);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(TestMapper.class);
+
+        job.setNumReduceTasks(0);
+
+        job.setInputFormatClass(TextInputFormat.class);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
+
+        job.setJarByClass(getClass());
+
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 1),
+                createJobInfo(job.getConfiguration()));
+
+        fut.get();
+
+        assertEquals(lineCnt, totalLineCnt.get());
+
+        assertEquals(32, taskWorkDirs.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapCombineRun() throws Exception {
+        int lineCnt = 10001;
+        String fileName = "/testFile";
+
+        prepareFile(fileName, lineCnt);
+
+        totalLineCnt.set(0);
+        taskWorkDirs.clear();
+
+        Configuration cfg = new Configuration();
+
+        cfg.setStrings("fs.igfs.impl", IgniteHadoopFileSystem.class.getName());
+        cfg.setBoolean(MAP_WRITE, true);
+
+        Job job = Job.getInstance(cfg);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(TestMapper.class);
+        job.setCombinerClass(TestCombiner.class);
+        job.setReducerClass(TestReducer.class);
+
+        job.setNumReduceTasks(2);
+
+        job.setInputFormatClass(TextInputFormat.class);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
+
+        job.setJarByClass(getClass());
+
+        HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 2);
+
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+
+        fut.get();
+
+        assertEquals(lineCnt, totalLineCnt.get());
+
+        assertEquals(34, taskWorkDirs.size());
+
+        for (int g = 0; g < gridCount(); g++)
+            grid(g).hadoop().finishFuture(jobId).get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapperException() throws Exception {
+        prepareFile("/testFile", 1000);
+
+        Configuration cfg = new Configuration();
+
+        cfg.setStrings("fs.igfs.impl", IgniteHadoopFileSystem.class.getName());
+
+        Job job = Job.getInstance(cfg);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(FailMapper.class);
+
+        job.setNumReduceTasks(0);
+
+        job.setInputFormatClass(TextInputFormat.class);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
+
+        job.setJarByClass(getClass());
+
+        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 3),
+                createJobInfo(job.getConfiguration()));
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fut.get();
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+    }
+
+    /**
+     * @param fileName File name.
+     * @param lineCnt Line count.
+     * @throws Exception If failed.
+     */
+    private void prepareFile(String fileName, int lineCnt) throws Exception {
+        IgniteFileSystem igfs = grid(0).fileSystem(igfsName);
+
+        try (OutputStream os = igfs.create(new IgfsPath(fileName), true)) {
+            PrintWriter w = new PrintWriter(new OutputStreamWriter(os));
+
+            for (int i = 0; i < lineCnt; i++)
+                w.print("Hello, Hadoop map-reduce!\n");
+
+            w.flush();
+        }
+    }
+
+    /**
+     * Prepare job with mappers to cancel.
+     * @return Fully configured job.
+     * @throws Exception If fails.
+     */
+    private Configuration prepareJobForCancelling() throws Exception {
+        prepareFile("/testFile", 1500);
+
+        executedTasks.set(0);
+        cancelledTasks.set(0);
+        failMapperId.set(0);
+        splitsCount.set(0);
+
+        Configuration cfg = new Configuration();
+
+        setupFileSystems(cfg);
+
+        Job job = Job.getInstance(cfg);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(CancellingTestMapper.class);
+
+        job.setNumReduceTasks(0);
+
+        job.setInputFormatClass(InFormat.class);
+
+        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
+        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
+
+        job.setJarByClass(getClass());
+
+        return job.getConfiguration();
+    }
+
+    /**
+     * Test input format.
+     */
+    private static class InFormat extends TextInputFormat {
+        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException {
+            List<InputSplit> res = super.getSplits(ctx);
+
+            splitsCount.set(res.size());
+
+            X.println("___ split of input: " + splitsCount.get());
+
+            return res;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTaskCancelling() throws Exception {
+        Configuration cfg = prepareJobForCancelling();
+
+        HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 1);
+
+        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(cfg));
+
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return splitsCount.get() > 0;
+            }
+        }, 20000)) {
+            U.dumpThreads(log);
+
+            assertTrue(false);
+        }
+
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return executedTasks.get() == splitsCount.get();
+            }
+        }, 20000)) {
+            U.dumpThreads(log);
+
+            assertTrue(false);
+        }
+
+        // Fail mapper with id "1", cancels others
+        failMapperId.set(1);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fut.get();
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+
+        assertEquals(executedTasks.get(), cancelledTasks.get() + 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJobKill() throws Exception {
+        Configuration cfg = prepareJobForCancelling();
+
+        Hadoop hadoop = grid(0).hadoop();
+
+        HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 1);
+
+        //Kill unknown job.
+        boolean killRes = hadoop.kill(jobId);
+
+        assertFalse(killRes);
+
+        final IgniteInternalFuture<?> fut = hadoop.submit(jobId, createJobInfo(cfg));
+
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return splitsCount.get() > 0;
+            }
+        }, 20000)) {
+            U.dumpThreads(log);
+
+            assertTrue(false);
+        }
+
+        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                X.println("___ executed tasks: " + executedTasks.get());
+
+                return executedTasks.get() == splitsCount.get();
+            }
+        }, 20000)) {
+            U.dumpThreads(log);
+
+            fail();
+        }
+
+        //Kill really ran job.
+        killRes = hadoop.kill(jobId);
+
+        assertTrue(killRes);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fut.get();
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+
+        assertEquals(executedTasks.get(), cancelledTasks.get());
+
+        //Kill the same job again.
+        killRes = hadoop.kill(jobId);
+
+        assertTrue(killRes);
+    }
+
+    private static class CancellingTestMapper extends Mapper<Object, Text, Text, IntWritable> {
+        private int mapperId;
+
+        /** {@inheritDoc} */
+        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
+            mapperId = executedTasks.incrementAndGet();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run(Context ctx) throws IOException, InterruptedException {
+            try {
+                super.run(ctx);
+            }
+            catch (HadoopTaskCancelledException e) {
+                cancelledTasks.incrementAndGet();
+
+                throw e;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            if (mapperId == failMapperId.get())
+                throw new IOException();
+
+            Thread.sleep(1000);
+        }
+    }
+
+    /**
+     * Test failing mapper.
+     */
+    private static class FailMapper extends Mapper<Object, Text, Text, IntWritable> {
+        /** {@inheritDoc} */
+        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            throw new IOException("Expected");
+        }
+    }
+
+    /**
+     * Mapper calculates number of lines.
+     */
+    private static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
+        /** Writable integer constant of '1'. */
+        private static final IntWritable ONE = new IntWritable(1);
+
+        /** Line count constant. */
+        public static final Text LINE_COUNT = new Text("lineCount");
+
+        /** {@inheritDoc} */
+        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
+            X.println("___ Mapper: " + ctx.getTaskAttemptID());
+
+            String taskId = ctx.getTaskAttemptID().toString();
+
+            LocalFileSystem locFs = FileSystem.getLocal(ctx.getConfiguration());
+
+            String workDir = locFs.getWorkingDirectory().toString();
+
+            assertNull(taskWorkDirs.put(workDir, taskId));
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            if (ctx.getConfiguration().getBoolean(MAP_WRITE, false))
+                ctx.write(LINE_COUNT, ONE);
+            else
+                totalLineCnt.incrementAndGet();
+        }
+    }
+
+    /**
+     * Combiner calculates number of lines.
+     */
+    private static class TestCombiner extends Reducer<Text, IntWritable, Text, IntWritable> {
+        /** */
+        IntWritable sum = new IntWritable();
+
+        /** {@inheritDoc} */
+        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
+            X.println("___ Combiner: ");
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
+            InterruptedException {
+            int lineCnt = 0;
+
+            for (IntWritable value : values)
+                lineCnt += value.get();
+
+            sum.set(lineCnt);
+
+            X.println("___ combo: " + lineCnt);
+
+            ctx.write(key, sum);
+        }
+    }
+
+    /**
+     * Combiner calculates number of lines.
+     */
+    private static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
+        /** */
+        IntWritable sum = new IntWritable();
+
+        /** {@inheritDoc} */
+        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
+            X.println("___ Reducer: " + ctx.getTaskAttemptID());
+
+            String taskId = ctx.getTaskAttemptID().toString();
+            String workDir = FileSystem.getLocal(ctx.getConfiguration()).getWorkingDirectory().toString();
+
+            assertNull(taskWorkDirs.put(workDir, taskId));
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
+            InterruptedException {
+            int lineCnt = 0;
+
+            for (IntWritable value : values) {
+                lineCnt += value.get();
+
+                X.println("___ rdcr: " + value.get());
+            }
+
+            sum.set(lineCnt);
+
+            ctx.write(key, sum);
+
+            X.println("___ RDCR SUM: " + lineCnt);
+
+            totalLineCnt.addAndGet(lineCnt);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
new file mode 100644
index 0000000..aaf0f92
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.google.common.base.*;
+import org.apache.hadoop.io.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ * Tests of Map, Combine and Reduce task executions of any version of hadoop API.
+ */
+abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
+    /** Empty hosts array. */
+    private static final String[] HOSTS = new String[0];
+
+    /**
+     * Creates some grid hadoop job. Override this method to create tests for any job implementation.
+     *
+     * @param inFile Input file name for the job.
+     * @param outFile Output file name for the job.
+     * @return Hadoop job.
+     * @throws IOException If fails.
+     */
+    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
+     */
+    public abstract String getOutputFileNamePrefix();
+
+    /**
+     * Tests map task execution.
+     *
+     * @throws Exception If fails.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testMapTask() throws Exception {
+        IgfsPath inDir = new IgfsPath(PATH_INPUT);
+
+        igfs.mkdirs(inDir);
+
+        IgfsPath inFile = new IgfsPath(inDir, HadoopWordCount2.class.getSimpleName() + "-input");
+
+        URI inFileUri = URI.create(igfsScheme() + inFile.toString());
+
+        try (PrintWriter pw = new PrintWriter(igfs.create(inFile, true))) {
+            pw.println("hello0 world0");
+            pw.println("world1 hello1");
+        }
+
+        HadoopFileBlock fileBlock1 = new HadoopFileBlock(HOSTS, inFileUri, 0, igfs.info(inFile).length() - 1);
+
+        try (PrintWriter pw = new PrintWriter(igfs.append(inFile, false))) {
+            pw.println("hello2 world2");
+            pw.println("world3 hello3");
+        }
+        HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, fileBlock1.length(),
+                igfs.info(inFile).length() - fileBlock1.length());
+
+        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
+
+        HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock1);
+
+        HadoopTestTaskContext ctx = new HadoopTestTaskContext(taskInfo, gridJob);
+
+        ctx.mockOutput().clear();
+
+        ctx.run();
+
+        assertEquals("hello0,1; world0,1; world1,1; hello1,1", Joiner.on("; ").join(ctx.mockOutput()));
+
+        ctx.mockOutput().clear();
+
+        ctx.taskInfo(new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock2));
+
+        ctx.run();
+
+        assertEquals("hello2,1; world2,1; world3,1; hello3,1", Joiner.on("; ").join(ctx.mockOutput()));
+    }
+
+    /**
+     * Generates input data for reduce-like operation into mock context input and runs the operation.
+     *
+     * @param gridJob Job is to create reduce task from.
+     * @param taskType Type of task - combine or reduce.
+     * @param taskNum Number of task in job.
+     * @param words Pairs of words and its counts.
+     * @return Context with mock output.
+     * @throws IgniteCheckedException If fails.
+     */
+    private HadoopTestTaskContext runTaskWithInput(HadoopV2Job gridJob, HadoopTaskType taskType,
+        int taskNum, String... words) throws IgniteCheckedException {
+        HadoopTaskInfo taskInfo = new HadoopTaskInfo(taskType, gridJob.id(), taskNum, 0, null);
+
+        HadoopTestTaskContext ctx = new HadoopTestTaskContext(taskInfo, gridJob);
+
+        for (int i = 0; i < words.length; i+=2) {
+            List<IntWritable> valList = new ArrayList<>();
+
+            for (int j = 0; j < Integer.parseInt(words[i + 1]); j++)
+                valList.add(new IntWritable(1));
+
+            ctx.mockInput().put(new Text(words[i]), valList);
+        }
+
+        ctx.run();
+
+        return ctx;
+    }
+
+    /**
+     * Tests reduce task execution.
+     *
+     * @throws Exception If fails.
+     */
+    public void testReduceTask() throws Exception {
+        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
+
+        runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 0, "word1", "5", "word2", "10");
+        runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 1, "word3", "7", "word4", "15");
+
+        assertEquals(
+            "word1\t5\n" +
+            "word2\t10\n",
+            readAndSortFile(PATH_OUTPUT + "/_temporary/0/task_00000000-0000-0000-0000-000000000000_0000_r_000000/" +
+                getOutputFileNamePrefix() + "00000")
+        );
+
+        assertEquals(
+            "word3\t7\n" +
+            "word4\t15\n",
+            readAndSortFile(PATH_OUTPUT + "/_temporary/0/task_00000000-0000-0000-0000-000000000000_0000_r_000001/" +
+                getOutputFileNamePrefix() + "00001")
+        );
+    }
+
+    /**
+     * Tests combine task execution.
+     *
+     * @throws Exception If fails.
+     */
+    public void testCombinerTask() throws Exception {
+        HadoopV2Job gridJob = getHadoopJob("/", "/");
+
+        HadoopTestTaskContext ctx =
+            runTaskWithInput(gridJob, HadoopTaskType.COMBINE, 0, "word1", "5", "word2", "10");
+
+        assertEquals("word1,5; word2,10", Joiner.on("; ").join(ctx.mockOutput()));
+
+        ctx = runTaskWithInput(gridJob, HadoopTaskType.COMBINE, 1, "word3", "7", "word4", "15");
+
+        assertEquals("word3,7; word4,15", Joiner.on("; ").join(ctx.mockOutput()));
+    }
+
+    /**
+     * Runs chain of map-combine task on file block.
+     *
+     * @param fileBlock block of input file to be processed.
+     * @param gridJob Hadoop job implementation.
+     * @return Context of combine task with mock output.
+     * @throws IgniteCheckedException If fails.
+     */
+    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopV2Job gridJob)
+        throws IgniteCheckedException {
+        HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock);
+
+        HadoopTestTaskContext mapCtx = new HadoopTestTaskContext(taskInfo, gridJob);
+
+        mapCtx.run();
+
+        //Prepare input for combine
+        taskInfo = new HadoopTaskInfo(HadoopTaskType.COMBINE, gridJob.id(), 0, 0, null);
+
+        HadoopTestTaskContext combineCtx = new HadoopTestTaskContext(taskInfo, gridJob);
+
+        combineCtx.makeTreeOfWritables(mapCtx.mockOutput());
+
+        combineCtx.run();
+
+        return combineCtx;
+    }
+
+    /**
+     * Tests all job in complex.
+     * Runs 2 chains of map-combine tasks and sends result into one reduce task.
+     *
+     * @throws Exception If fails.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testAllTasks() throws Exception {
+        IgfsPath inDir = new IgfsPath(PATH_INPUT);
+
+        igfs.mkdirs(inDir);
+
+        IgfsPath inFile = new IgfsPath(inDir, HadoopWordCount2.class.getSimpleName() + "-input");
+
+        URI inFileUri = URI.create(igfsScheme() + inFile.toString());
+
+        generateTestFile(inFile.toString(), "red", 100, "blue", 200, "green", 150, "yellow", 70);
+
+        //Split file into two blocks
+        long fileLen = igfs.info(inFile).length();
+
+        Long l = fileLen / 2;
+
+        HadoopFileBlock fileBlock1 = new HadoopFileBlock(HOSTS, inFileUri, 0, l);
+        HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, l, fileLen - l);
+
+        HadoopV2Job gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
+
+        HadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
+
+        HadoopTestTaskContext combine2Ctx = runMapCombineTask(fileBlock2, gridJob);
+
+        //Prepare input for combine
+        HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.REDUCE, gridJob.id(), 0, 0, null);
+
+        HadoopTestTaskContext reduceCtx = new HadoopTestTaskContext(taskInfo, gridJob);
+
+        reduceCtx.makeTreeOfWritables(combine1Ctx.mockOutput());
+        reduceCtx.makeTreeOfWritables(combine2Ctx.mockOutput());
+
+        reduceCtx.run();
+
+        reduceCtx.taskInfo(new HadoopTaskInfo(HadoopTaskType.COMMIT, gridJob.id(), 0, 0, null));
+
+        reduceCtx.run();
+
+        assertEquals(
+            "blue\t200\n" +
+            "green\t150\n" +
+            "red\t100\n" +
+            "yellow\t70\n",
+            readAndSortFile(PATH_OUTPUT + "/" + getOutputFileNamePrefix() + "00000")
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
new file mode 100644
index 0000000..b41a260
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.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;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+import java.util.*;
+
+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.
+ */
+public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
+    /**
+     * Creates WordCount hadoop job for API v1.
+     *
+     * @param inFile Input file name for the job.
+     * @param outFile Output file name for the job.
+     * @return Hadoop job.
+     * @throws IOException If fails.
+     */
+    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+        JobConf jobConf = HadoopWordCount1.getJob(inFile, outFile);
+
+        setupFileSystems(jobConf);
+
+        HadoopDefaultJobInfo jobInfo = createJobInfo(jobConf);
+
+        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+
+        return new HadoopV2Job(jobId, jobInfo, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getOutputFileNamePrefix() {
+        return "part-";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
new file mode 100644
index 0000000..b677c63
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.util.*;
+
+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.
+ */
+public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
+    /**
+     * Creates WordCount hadoop job for API v2.
+     *
+     * @param inFile Input file name for the job.
+     * @param outFile Output file name for the job.
+     * @return Hadoop job.
+     * @throws Exception if fails.
+     */
+    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+        Job job = Job.getInstance();
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        HadoopWordCount2.setTasksClasses(job, true, true, true);
+
+        Configuration conf = job.getConfiguration();
+
+        setupFileSystems(conf);
+
+        FileInputFormat.setInputPaths(job, new Path(inFile));
+        FileOutputFormat.setOutputPath(job, new Path(outFile));
+
+        job.setJarByClass(HadoopWordCount2.class);
+
+        Job hadoopJob = HadoopWordCount2.getJob(inFile, outFile);
+
+        HadoopDefaultJobInfo jobInfo = createJobInfo(hadoopJob.getConfiguration());
+
+        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+
+        return new HadoopV2Job(jobId, jobInfo, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getOutputFileNamePrefix() {
+        return "part-r-";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java
new file mode 100644
index 0000000..a56c7c7
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.processors.hadoop.planner.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Round-robin mr planner.
+ */
+public class HadoopTestRoundRobinMrPlanner implements HadoopMapReducePlanner {
+    /** {@inheritDoc} */
+    @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
+        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException {
+        if (top.isEmpty())
+            throw new IllegalArgumentException("Topology is empty");
+
+        // Has at least one element.
+        Iterator<ClusterNode> it = top.iterator();
+
+        Map<UUID, Collection<HadoopInputSplit>> mappers = new HashMap<>();
+
+        for (HadoopInputSplit block : job.input()) {
+            ClusterNode node = it.next();
+
+            Collection<HadoopInputSplit> nodeBlocks = mappers.get(node.id());
+
+            if (nodeBlocks == null) {
+                nodeBlocks = new ArrayList<>();
+
+                mappers.put(node.id(), nodeBlocks);
+            }
+
+            nodeBlocks.add(block);
+
+            if (!it.hasNext())
+                it = top.iterator();
+        }
+
+        int[] rdc = new int[job.info().reducers()];
+
+        for (int i = 0; i < rdc.length; i++)
+            rdc[i] = i;
+
+        return new HadoopDefaultMapReducePlan(mappers, Collections.singletonMap(it.next().id(), rdc));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..e444270
--- /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 HadoopTaskOutput output = new HadoopTaskOutput() {
+        /** {@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 HadoopTaskInput input = new HadoopTaskInput() {
+        /** 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(HadoopTaskInfo taskInfo, HadoopJob 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(HadoopJob 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 HadoopTaskOutput output() {
+        return output;
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopTaskInput input() {
+        return input;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java
new file mode 100644
index 0000000..ef60762
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.io.*;
+import java.util.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * Utility class for tests.
+ */
+public class HadoopTestUtils {
+    /**
+     * Checks that job statistics file contains valid strings only.
+     *
+     * @param reader Buffered reader to get lines of job statistics.
+     * @return Amount of events.
+     * @throws IOException If failed.
+     */
+    public static long simpleCheckJobStatFile(BufferedReader reader) throws IOException {
+        Collection<String> phases = new HashSet<>();
+
+        phases.add("submit");
+        phases.add("prepare");
+        phases.add("start");
+        phases.add("finish");
+        phases.add("requestId");
+        phases.add("responseId");
+
+        Collection<String> evtTypes = new HashSet<>();
+
+        evtTypes.add("JOB");
+        evtTypes.add("SETUP");
+        evtTypes.add("MAP");
+        evtTypes.add("SHUFFLE");
+        evtTypes.add("REDUCE");
+        evtTypes.add("COMBINE");
+        evtTypes.add("COMMIT");
+
+        long evtCnt = 0;
+        String line;
+
+        Map<Long, String> reduceNodes = new HashMap<>();
+
+        while((line = reader.readLine()) != null) {
+            String[] splitLine = line.split(":");
+
+            //Try parse timestamp
+            Long.parseLong(splitLine[1]);
+
+            String[] evt = splitLine[0].split(" ");
+
+            assertTrue("Unknown event '" + evt[0] + "'", evtTypes.contains(evt[0]));
+
+            String phase;
+
+            if ("JOB".equals(evt[0]))
+                phase = evt[1];
+            else {
+                assertEquals(4, evt.length);
+                assertTrue("The node id is not defined", !F.isEmpty(evt[3]));
+
+                long taskNum = Long.parseLong(evt[1]);
+
+                if (("REDUCE".equals(evt[0]) || "SHUFFLE".equals(evt[0]))) {
+                    String nodeId = reduceNodes.get(taskNum);
+
+                    if (nodeId == null)
+                        reduceNodes.put(taskNum, evt[3]);
+                    else
+                        assertEquals("Different nodes for SHUFFLE and REDUCE tasks", nodeId, evt[3]);
+                }
+
+                phase = evt[2];
+            }
+
+            assertTrue("Unknown phase '" + phase + "' in " + Arrays.toString(evt), phases.contains(phase));
+
+            evtCnt++;
+        }
+
+        return evtCnt;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..ebc89f4
--- /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 HadoopAbstractSelfTest {
+    /** */
+    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 HadoopJob} 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());
+
+        HadoopJob job = new HadoopV2Job(new HadoopJobId(UUID.randomUUID(), 1), createJobInfo(cfg), log);
+
+        HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
+            null));
+
+        HadoopSerialization 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());
+    }
+}


[22/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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);
-    }
-}


[31/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..351839a
--- /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(HadoopCounter 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(HadoopTaskInfo 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(HadoopTaskInfo 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(HadoopTaskInfo 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(HadoopTaskInfo info, long ts) {
+        if (info.type() == HadoopTaskType.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(HadoopTaskInfo 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(HadoopJobInfo 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, HadoopJobInfo 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 HadoopCounters object.
+     *
+     * @param cntrs HadoopCounters 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(HadoopCounters 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/6423cf02/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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 52e7d29..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.igfs.hadoop.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 IgfsHadoopFileSystem)
-            ((IgfsHadoopFileSystem)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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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..509f443
--- /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.FileSystemConfiguration.*;
+
+/**
+ * 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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java
new file mode 100644
index 0000000..7edcec0
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.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.processors.hadoop.fs;
+
+/**
+ * This class lists parameters that can be specified in Hadoop configuration.
+ * Hadoop configuration can be specified in {@code core-site.xml} file
+ * or passed to map-reduce task directly when using Hadoop driver for IGFS file system:
+ * <ul>
+ *     <li>
+ *         {@code fs.igfs.[name].open.sequential_reads_before_prefetch} - this parameter overrides
+ *         the one specified in {@link org.apache.ignite.configuration.FileSystemConfiguration#getSequentialReadsBeforePrefetch()}
+ *         IGFS data node configuration property.
+ *     </li>
+ *     <li>
+ *         {@code fs.igfs.[name].log.enabled} - specifies whether IGFS sampling logger is enabled. If
+ *         {@code true}, then all file system operations will be logged to a file.
+ *     </li>
+ *     <li>{@code fs.igfs.[name].log.dir} - specifies log directory where sampling log files should be placed.</li>
+ *     <li>
+ *         {@code fs.igfs.[name].log.batch_size} - specifies how many log entries are accumulated in a batch before
+ *         it gets flushed to log file. Higher values will imply greater performance, but will increase delay
+ *         before record appears in the log file.
+ *     </li>
+ *     <li>
+ *         {@code fs.igfs.[name].colocated.writes} - specifies whether written files should be colocated on data
+ *         node to which client is connected. If {@code true}, file will not be distributed and will be written
+ *         to a single data node. Default value is {@code true}.
+ *     </li>
+ *     <li>
+ *         {@code fs.igfs.prefer.local.writes} - specifies whether file preferably should be written to
+ *         local data node if it has enough free space. After some time it can be redistributed across nodes though.
+ *     </li>
+ * </ul>
+ * Where {@code [name]} is file system endpoint which you specify in file system URI authority part. E.g. in
+ * case your file system URI is {@code igfs://127.0.0.1:10500} then {@code name} will be {@code 127.0.0.1:10500}.
+ * <p>
+ * Sample configuration that can be placed to {@code core-site.xml} file:
+ * <pre name="code" class="xml">
+ *     &lt;property&gt;
+ *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.enabled&lt;/name&gt;
+ *         &lt;value&gt;true&lt;/value&gt;
+ *     &lt;/property&gt;
+ *     &lt;property&gt;
+ *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.dir&lt;/name&gt;
+ *         &lt;value&gt;/home/apache/ignite/log/sampling&lt;/value&gt;
+ *     &lt;/property&gt;
+ *     &lt;property&gt;
+ *         &lt;name&gt;fs.igfs.127.0.0.1:10500.log.batch_size&lt;/name&gt;
+ *         &lt;value&gt;16&lt;/value&gt;
+ *     &lt;/property&gt;
+ * </pre>
+ * Parameters could also be specified per mapreduce job, e.g.
+ * <pre name="code" class="bash">
+ * hadoop jar myjarfile.jar MyMapReduceJob -Dfs.igfs.open.sequential_reads_before_prefetch=4
+ * </pre>
+ * If you want to use these parameters in code, then you have to substitute you file system name in it. The easiest
+ * way to do that is {@code String.format(PARAM_IGFS_COLOCATED_WRITES, [name])}.
+ */
+public class HadoopParameters {
+    /** Parameter name for control over file colocation write mode. */
+    public static final String PARAM_IGFS_COLOCATED_WRITES = "fs.igfs.%s.colocated.writes";
+
+    /** Parameter name for custom sequential reads before prefetch value. */
+    public static final String PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH =
+        "fs.igfs.%s.open.sequential_reads_before_prefetch";
+
+    /** Parameter name for client logger directory. */
+    public static final String PARAM_IGFS_LOG_DIR = "fs.igfs.%s.log.dir";
+
+    /** Parameter name for log batch size. */
+    public static final String PARAM_IGFS_LOG_BATCH_SIZE = "fs.igfs.%s.log.batch_size";
+
+    /** Parameter name for log enabled flag. */
+    public static final String PARAM_IGFS_LOG_ENABLED = "fs.igfs.%s.log.enabled";
+
+    /** Parameter name for prefer local writes flag. */
+    public static final String PARAM_IGFS_PREFER_LOCAL_WRITES = "fs.igfs.prefer.local.writes";
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java
new file mode 100644
index 0000000..b3cb235
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java
new file mode 100644
index 0000000..ff69478
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEndpoint.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEndpoint.java
new file mode 100644
index 0000000..7502f57
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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.FileSystemConfiguration.*;
+
+/**
+ * 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(IgniteFileSystem.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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
new file mode 100644
index 0000000..2200e78
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java
new file mode 100644
index 0000000..59a8f49
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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;
+    }
+}


[07/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 8319255..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java
+++ /dev/null
@@ -1,222 +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.ignite.configuration.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
-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.junits.common.*;
-
-import java.io.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * Abstract class for Hadoop tests.
- */
-public abstract class GridHadoopAbstractSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** REST port. */
-    protected static final int REST_PORT = 11212;
-
-    /** IGFS name. */
-    protected static final String igfsName = null;
-
-    /** IGFS name. */
-    protected static final String igfsMetaCacheName = "meta";
-
-    /** IGFS name. */
-    protected static final String igfsDataCacheName = "data";
-
-    /** IGFS block size. */
-    protected static final int igfsBlockSize = 1024;
-
-    /** IGFS block group size. */
-    protected static final int igfsBlockGroupSize = 8;
-
-    /** Initial REST port. */
-    private int restPort = REST_PORT;
-
-    /** Initial classpath. */
-    private static String initCp;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        // Add surefire classpath to regular classpath.
-        initCp = System.getProperty("java.class.path");
-
-        String surefireCp = System.getProperty("surefire.test.class.path");
-
-        if (surefireCp != null)
-            System.setProperty("java.class.path", initCp + File.pathSeparatorChar + surefireCp);
-
-        super.beforeTestsStarted();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        // Restore classpath.
-        System.setProperty("java.class.path", initCp);
-
-        initCp = null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setHadoopConfiguration(hadoopConfiguration(gridName));
-
-        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
-
-        commSpi.setSharedMemoryPort(-1);
-
-        cfg.setCommunicationSpi(commSpi);
-
-        TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        if (igfsEnabled()) {
-            cfg.setCacheConfiguration(metaCacheConfiguration(), dataCacheConfiguration());
-
-            cfg.setIgfsConfiguration(igfsConfiguration());
-        }
-
-        if (restEnabled()) {
-            ConnectorConfiguration clnCfg = new ConnectorConfiguration();
-
-            clnCfg.setPort(restPort++);
-
-            cfg.setConnectorConfiguration(clnCfg);
-        }
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setPeerClassLoadingEnabled(false);
-
-        return cfg;
-    }
-
-    /**
-     * @param gridName Grid name.
-     * @return Hadoop configuration.
-     */
-    public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = new GridHadoopConfiguration();
-
-        cfg.setMaxParallelTasks(3);
-
-        return cfg;
-    }
-
-    /**
-     * @return IGFS configuration.
-     */
-    public IgfsConfiguration igfsConfiguration() {
-        IgfsConfiguration cfg = new IgfsConfiguration();
-
-        cfg.setName(igfsName);
-        cfg.setBlockSize(igfsBlockSize);
-        cfg.setDataCacheName(igfsDataCacheName);
-        cfg.setMetaCacheName(igfsMetaCacheName);
-        cfg.setFragmentizerEnabled(false);
-
-        return cfg;
-    }
-
-    /**
-     * @return IGFS meta cache configuration.
-     */
-    public CacheConfiguration metaCacheConfiguration() {
-        CacheConfiguration cfg = new CacheConfiguration();
-
-        cfg.setName(igfsMetaCacheName);
-        cfg.setCacheMode(REPLICATED);
-        cfg.setAtomicityMode(TRANSACTIONAL);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        return cfg;
-    }
-
-    /**
-     * @return IGFS data cache configuration.
-     */
-    private CacheConfiguration dataCacheConfiguration() {
-        CacheConfiguration cfg = new CacheConfiguration();
-
-        cfg.setName(igfsDataCacheName);
-        cfg.setCacheMode(PARTITIONED);
-        cfg.setAtomicityMode(TRANSACTIONAL);
-        cfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(igfsBlockGroupSize));
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        return cfg;
-    }
-
-    /**
-     * @return {@code True} if IGFS is enabled on Hadoop nodes.
-     */
-    protected boolean igfsEnabled() {
-        return false;
-    }
-
-    /**
-     * @return {@code True} if REST is enabled on Hadoop nodes.
-     */
-    protected boolean restEnabled() {
-        return false;
-    }
-
-    /**
-     * @return Number of nodes to start.
-     */
-    protected int gridCount() {
-        return 3;
-    }
-
-    /**
-     * @param cfg Config.
-     */
-    protected void setupFileSystems(Configuration cfg) {
-        cfg.set("fs.defaultFS", igfsScheme());
-        cfg.set("fs.igfs.impl", org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem.class.getName());
-        cfg.set("fs.AbstractFileSystem.igfs.impl", IgfsHadoopFileSystem.
-            class.getName());
-
-        GridHadoopFileSystemsUtils.setupFileSystems(cfg);
-    }
-
-    /**
-     * @return IGFS scheme for test.
-     */
-    protected String igfsScheme() {
-        return "igfs://:" + getTestGridName(0) + "@/";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractWordCountTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractWordCountTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractWordCountTest.java
deleted file mode 100644
index ebbc0a6..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractWordCountTest.java
+++ /dev/null
@@ -1,138 +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 com.google.common.base.*;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Abstract class for tests based on WordCount test job.
- */
-public abstract class GridHadoopAbstractWordCountTest extends GridHadoopAbstractSelfTest {
-    /** Input path. */
-    protected static final String PATH_INPUT = "/input";
-
-    /** Output path. */
-    protected static final String PATH_OUTPUT = "/output";
-
-    /** IGFS instance. */
-    protected IgfsEx igfs;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        Configuration cfg = new Configuration();
-
-        setupFileSystems(cfg);
-
-        // Init cache by correct LocalFileSystem implementation
-        FileSystem.getLocal(cfg);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        igfs = (IgfsEx)startGrids(gridCount()).fileSystem(igfsName);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /**
-     * Generates test file.
-     *
-     * @param path File name.
-     * @param wordCounts Words and counts.
-     * @throws Exception If failed.
-     */
-    protected void generateTestFile(String path, Object... wordCounts) throws Exception {
-        List<String> wordsArr = new ArrayList<>();
-
-        //Generating
-        for (int i = 0; i < wordCounts.length; i += 2) {
-            String word = (String) wordCounts[i];
-            int cnt = (Integer) wordCounts[i + 1];
-
-            while (cnt-- > 0)
-                wordsArr.add(word);
-        }
-
-        //Shuffling
-        for (int i = 0; i < wordsArr.size(); i++) {
-            int j = (int)(Math.random() * wordsArr.size());
-
-            Collections.swap(wordsArr, i, j);
-        }
-
-        //Input file preparing
-        PrintWriter testInputFileWriter = new PrintWriter(igfs.create(new IgfsPath(path), true));
-
-        int j = 0;
-
-        while (j < wordsArr.size()) {
-            int i = 5 + (int)(Math.random() * 5);
-
-            List<String> subList = wordsArr.subList(j, Math.min(j + i, wordsArr.size()));
-            j += i;
-
-            testInputFileWriter.println(Joiner.on(' ').join(subList));
-        }
-
-        testInputFileWriter.close();
-    }
-
-    /**
-     * Reads whole text file into String.
-     *
-     * @param fileName Name of the file to read.
-     * @return Content of the file as String value.
-     * @throws Exception If could not read the file.
-     */
-    protected String readAndSortFile(String fileName) throws Exception {
-        BufferedReader reader = new BufferedReader(new InputStreamReader(igfs.open(new IgfsPath(fileName))));
-
-        List<String> list = new ArrayList<>();
-
-        String line;
-
-        while ((line = reader.readLine()) != null)
-            list.add(line);
-
-        Collections.sort(list);
-
-        return Joiner.on('\n').join(list) + "\n";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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
deleted file mode 100644
index 80cd226..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.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;
-
-import com.google.common.base.*;
-import org.apache.ignite.*;
-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.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.jdk8.backport.*;
-
-import java.io.*;
-import java.nio.file.*;
-import java.util.*;
-
-/**
- * Test of integration with Hadoop client via command line interface.
- */
-public class GridHadoopCommandLineTest extends GridCommonAbstractTest {
-    /** IGFS instance. */
-    private IgfsEx igfs;
-
-    /** */
-    private static final String igfsName = "igfs";
-
-    /** */
-    private static File testWorkDir;
-
-    /** */
-    private static String hadoopHome;
-
-    /** */
-    private static String hiveHome;
-
-    /** */
-    private static File examplesJar;
-
-    /**
-     *
-     * @param path File name.
-     * @param wordCounts Words and counts.
-     * @throws Exception If failed.
-     */
-    private void generateTestFile(File path, Object... wordCounts) throws Exception {
-        List<String> wordsArr = new ArrayList<>();
-
-        //Generating
-        for (int i = 0; i < wordCounts.length; i += 2) {
-            String word = (String) wordCounts[i];
-            int cnt = (Integer) wordCounts[i + 1];
-
-            while (cnt-- > 0)
-                wordsArr.add(word);
-        }
-
-        //Shuffling
-        for (int i = 0; i < wordsArr.size(); i++) {
-            int j = (int)(Math.random() * wordsArr.size());
-
-            Collections.swap(wordsArr, i, j);
-        }
-
-        //Writing file
-        try (PrintWriter writer = new PrintWriter(path)) {
-            int j = 0;
-
-            while (j < wordsArr.size()) {
-                int i = 5 + (int)(Math.random() * 5);
-
-                List<String> subList = wordsArr.subList(j, Math.min(j + i, wordsArr.size()));
-                j += i;
-
-                writer.println(Joiner.on(' ').join(subList));
-            }
-
-            writer.flush();
-        }
-    }
-
-    /**
-     * Generates two data files to join its with Hive.
-     *
-     * @throws FileNotFoundException If failed.
-     */
-    private void generateHiveTestFiles() throws FileNotFoundException {
-        try (PrintWriter writerA = new PrintWriter(new File(testWorkDir, "data-a"));
-             PrintWriter writerB = new PrintWriter(new File(testWorkDir, "data-b"))) {
-            char sep = '\t';
-
-            int idB = 0;
-            int idA = 0;
-            int v = 1000;
-
-            for (int i = 0; i < 1000; i++) {
-                writerA.print(idA++);
-                writerA.print(sep);
-                writerA.println(idB);
-
-                writerB.print(idB++);
-                writerB.print(sep);
-                writerB.println(v += 2);
-
-                writerB.print(idB++);
-                writerB.print(sep);
-                writerB.println(v += 2);
-            }
-
-            writerA.flush();
-            writerB.flush();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        hiveHome = IgniteSystemProperties.getString("HIVE_HOME");
-
-        assertFalse("HIVE_HOME hasn't been set.", F.isEmpty(hiveHome));
-
-        hadoopHome = IgniteSystemProperties.getString("HADOOP_HOME");
-
-        assertFalse("HADOOP_HOME hasn't been set.", F.isEmpty(hadoopHome));
-
-        String mapredHome = hadoopHome + "/share/hadoop/mapreduce";
-
-        File[] fileList = new File(mapredHome).listFiles(new FileFilter() {
-            @Override public boolean accept(File pathname) {
-                return pathname.getName().startsWith("hadoop-mapreduce-examples-") &&
-                    pathname.getName().endsWith(".jar");
-            }
-        });
-
-        assertEquals("Invalid hadoop distribution.", 1, fileList.length);
-
-        examplesJar = fileList[0];
-
-        testWorkDir = Files.createTempDirectory("hadoop-cli-test").toFile();
-
-        U.copy(U.resolveIgnitePath("docs/core-site.ignite.xml"), new File(testWorkDir, "core-site.xml"), false);
-
-        File srcFile = U.resolveIgnitePath("docs/mapred-site.ignite.xml");
-        File dstFile = new File(testWorkDir, "mapred-site.xml");
-
-        try (BufferedReader in = new BufferedReader(new FileReader(srcFile));
-             PrintWriter out = new PrintWriter(dstFile)) {
-            String line;
-
-            while ((line = in.readLine()) != null) {
-                if (line.startsWith("</configuration>"))
-                    out.println(
-                        "    <property>\n" +
-                        "        <name>" + GridHadoopUtils.JOB_COUNTER_WRITER_PROPERTY + "</name>\n" +
-                        "        <value>" + GridHadoopFSCounterWriter.class.getName() + "</value>\n" +
-                        "    </property>\n");
-
-                out.println(line);
-            }
-
-            out.flush();
-        }
-
-        generateTestFile(new File(testWorkDir, "test-data"), "red", 100, "green", 200, "blue", 150, "yellow", 50);
-
-        generateHiveTestFiles();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        U.delete(testWorkDir);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        igfs = (IgfsEx) Ignition.start("config/hadoop/default-config.xml").fileSystem(igfsName);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /**
-     * Creates the process build with appropriate environment to run Hadoop CLI.
-     *
-     * @return Process builder.
-     */
-    private ProcessBuilder createProcessBuilder() {
-        String sep = ":";
-
-        String ggClsPath = GridHadoopJob.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
-            GridHadoopJobTracker.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
-            ConcurrentHashMap8.class.getProtectionDomain().getCodeSource().getLocation().getPath();
-
-        ProcessBuilder res = new ProcessBuilder();
-
-        res.environment().put("HADOOP_HOME", hadoopHome);
-        res.environment().put("HADOOP_CLASSPATH", ggClsPath);
-        res.environment().put("HADOOP_CONF_DIR", testWorkDir.getAbsolutePath());
-
-        res.redirectErrorStream(true);
-
-        return res;
-    }
-
-    /**
-     * Waits for process exit and prints the its output.
-     *
-     * @param proc Process.
-     * @return Exit code.
-     * @throws Exception If failed.
-     */
-    private int watchProcess(Process proc) throws Exception {
-        BufferedReader reader = new BufferedReader(new InputStreamReader(proc.getInputStream()));
-
-        String line;
-
-        while ((line = reader.readLine()) != null)
-            log().info(line);
-
-        return proc.waitFor();
-    }
-
-    /**
-     * Executes Hadoop command line tool.
-     *
-     * @param args Arguments for Hadoop command line tool.
-     * @return Process exit code.
-     * @throws Exception If failed.
-     */
-    private int executeHadoopCmd(String... args) throws Exception {
-        ProcessBuilder procBuilder = createProcessBuilder();
-
-        List<String> cmd = new ArrayList<>();
-
-        cmd.add(hadoopHome + "/bin/hadoop");
-        cmd.addAll(Arrays.asList(args));
-
-        procBuilder.command(cmd);
-
-        log().info("Execute: " + procBuilder.command());
-
-        return watchProcess(procBuilder.start());
-    }
-
-    /**
-     * Executes Hive query.
-     *
-     * @param qry Query.
-     * @return Process exit code.
-     * @throws Exception If failed.
-     */
-    private int executeHiveQuery(String qry) throws Exception {
-        ProcessBuilder procBuilder = createProcessBuilder();
-
-        List<String> cmd = new ArrayList<>();
-
-        procBuilder.command(cmd);
-
-        cmd.add(hiveHome + "/bin/hive");
-
-        cmd.add("--hiveconf");
-        cmd.add("hive.rpc.query.plan=true");
-
-        cmd.add("--hiveconf");
-        cmd.add("javax.jdo.option.ConnectionURL=jdbc:derby:" + testWorkDir.getAbsolutePath() + "/metastore_db;" +
-            "databaseName=metastore_db;create=true");
-
-        cmd.add("-e");
-        cmd.add(qry);
-
-        procBuilder.command(cmd);
-
-        log().info("Execute: " + procBuilder.command());
-
-        return watchProcess(procBuilder.start());
-    }
-
-    /**
-     * Tests Hadoop command line integration.
-     */
-    public void testHadoopCommandLine() throws Exception {
-        assertEquals(0, executeHadoopCmd("fs", "-ls", "/"));
-
-        assertEquals(0, executeHadoopCmd("fs", "-mkdir", "/input"));
-
-        assertEquals(0, executeHadoopCmd("fs", "-put", new File(testWorkDir, "test-data").getAbsolutePath(), "/input"));
-
-        assertTrue(igfs.exists(new IgfsPath("/input/test-data")));
-
-        assertEquals(0, executeHadoopCmd("jar", examplesJar.getAbsolutePath(), "wordcount", "/input", "/output"));
-
-        IgfsPath path = new IgfsPath("/user/" + System.getProperty("user.name") + "/");
-
-        assertTrue(igfs.exists(path));
-
-        IgfsPath jobStatPath = null;
-
-        for (IgfsPath jobPath : igfs.listPaths(path)) {
-            assertNull(jobStatPath);
-
-            jobStatPath = jobPath;
-        }
-
-        File locStatFile = new File(testWorkDir, "performance");
-
-        assertEquals(0, executeHadoopCmd("fs", "-get", jobStatPath.toString() + "/performance", locStatFile.toString()));
-
-        long evtCnt = GridHadoopTestUtils.simpleCheckJobStatFile(new BufferedReader(new FileReader(locStatFile)));
-
-        assertTrue(evtCnt >= 22); //It's the minimum amount of events for job with combiner.
-
-        assertTrue(igfs.exists(new IgfsPath("/output")));
-
-        BufferedReader in = new BufferedReader(new InputStreamReader(igfs.open(new IgfsPath("/output/part-r-00000"))));
-
-        List<String> res = new ArrayList<>();
-
-        String line;
-
-        while ((line = in.readLine()) != null)
-            res.add(line);
-
-        Collections.sort(res);
-
-        assertEquals("[blue\t150, green\t200, red\t100, yellow\t50]", res.toString());
-    }
-
-    /**
-     * Runs query check result.
-     *
-     * @param expRes Expected result.
-     * @param qry Query.
-     * @throws Exception If failed.
-     */
-    private void checkQuery(String expRes, String qry) throws Exception {
-        assertEquals(0, executeHiveQuery("drop table if exists result"));
-
-        assertEquals(0, executeHiveQuery(
-            "create table result " +
-            "row format delimited fields terminated by ' ' " +
-            "stored as textfile " +
-            "location '/result' as " + qry
-        ));
-
-        IgfsInputStreamAdapter in = igfs.open(new IgfsPath("/result/000000_0"));
-
-        byte[] buf = new byte[(int) in.length()];
-
-        in.read(buf);
-
-        assertEquals(expRes, new String(buf));
-    }
-
-    /**
-     * Tests Hive integration.
-     */
-    public void testHiveCommandLine() throws Exception {
-        assertEquals(0, executeHiveQuery(
-            "create table table_a (" +
-                "id_a int," +
-                "id_b int" +
-            ") " +
-            "row format delimited fields terminated by '\\t'" +
-            "stored as textfile " +
-            "location '/table-a'"
-        ));
-
-        assertEquals(0, executeHadoopCmd("fs", "-put", new File(testWorkDir, "data-a").getAbsolutePath(), "/table-a"));
-
-        assertEquals(0, executeHiveQuery(
-            "create table table_b (" +
-                "id_b int," +
-                "rndv int" +
-            ") " +
-            "row format delimited fields terminated by '\\t'" +
-            "stored as textfile " +
-            "location '/table-b'"
-        ));
-
-        assertEquals(0, executeHadoopCmd("fs", "-put", new File(testWorkDir, "data-b").getAbsolutePath(), "/table-b"));
-
-        checkQuery(
-            "0 0\n" +
-            "1 2\n" +
-            "2 4\n" +
-            "3 6\n" +
-            "4 8\n" +
-            "5 10\n" +
-            "6 12\n" +
-            "7 14\n" +
-            "8 16\n" +
-            "9 18\n",
-            "select * from table_a order by id_a limit 10"
-        );
-
-        checkQuery("2000\n", "select count(id_b) from table_b");
-
-        checkQuery(
-            "250 500 2002\n" +
-            "251 502 2006\n" +
-            "252 504 2010\n" +
-            "253 506 2014\n" +
-            "254 508 2018\n" +
-            "255 510 2022\n" +
-            "256 512 2026\n" +
-            "257 514 2030\n" +
-            "258 516 2034\n" +
-            "259 518 2038\n",
-            "select a.id_a, a.id_b, b.rndv" +
-            " from table_a a" +
-            " inner join table_b b on a.id_b = b.id_b" +
-            " where b.rndv > 2000" +
-            " order by a.id_a limit 10"
-        );
-
-        checkQuery("1000\n", "select count(b.id_b) from table_a a inner join table_b b on a.id_b = b.id_b");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 b1b0275..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 GridHadoopDefaultJobInfo() {
-                @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/6423cf02/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
deleted file mode 100644
index 18e5c03..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java
+++ /dev/null
@@ -1,177 +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.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
-import org.apache.ignite.testframework.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Test file systems for the working directory multi-threading support.
- */
-public class GridHadoopFileSystemsTest extends GridHadoopAbstractSelfTest {
-    private static final int THREAD_COUNT = 3;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-
-    /**
-     * Test the file system with specified URI for the multi-thread working directory support.
-     *
-     * @param uri Base URI of the file system (scheme and authority).
-     * @throws Exception If fails.
-     */
-    private void testFileSystem(final URI uri) throws Exception {
-        final Configuration cfg = new Configuration();
-
-        setupFileSystems(cfg);
-
-        cfg.set(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP,
-            new Path(new Path(uri), "user/" + System.getProperty("user.name")).toString());
-
-        final CountDownLatch changeUserPhase = new CountDownLatch(THREAD_COUNT);
-        final CountDownLatch changeDirPhase = new CountDownLatch(THREAD_COUNT);
-        final CountDownLatch changeAbsDirPhase = new CountDownLatch(THREAD_COUNT);
-        final CountDownLatch finishPhase = new CountDownLatch(THREAD_COUNT);
-
-        final Path[] newUserInitWorkDir = new Path[THREAD_COUNT];
-        final Path[] newWorkDir = new Path[THREAD_COUNT];
-        final Path[] newAbsWorkDir = new Path[THREAD_COUNT];
-        final Path[] newInstanceWorkDir = new Path[THREAD_COUNT];
-
-        final AtomicInteger threadNum = new AtomicInteger(0);
-
-        GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                try {
-                    int curThreadNum = threadNum.getAndIncrement();
-
-                    FileSystem fs = FileSystem.get(uri, cfg);
-
-                    GridHadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
-
-                    if ("file".equals(uri.getScheme()))
-                        FileSystem.get(uri, cfg).setWorkingDirectory(new Path("file:///user/user" + curThreadNum));
-
-                    changeUserPhase.countDown();
-                    changeUserPhase.await();
-
-                    newUserInitWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
-
-                    FileSystem.get(uri, cfg).setWorkingDirectory(new Path("folder" + curThreadNum));
-
-                    changeDirPhase.countDown();
-                    changeDirPhase.await();
-
-                    newWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
-
-                    FileSystem.get(uri, cfg).setWorkingDirectory(new Path("/folder" + curThreadNum));
-
-                    changeAbsDirPhase.countDown();
-                    changeAbsDirPhase.await();
-
-                    newAbsWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
-
-                    newInstanceWorkDir[curThreadNum] = FileSystem.newInstance(uri, cfg).getWorkingDirectory();
-
-                    finishPhase.countDown();
-                }
-                catch (InterruptedException | IOException e) {
-                    error("Failed to execute test thread.", e);
-
-                    fail();
-                }
-            }
-        }, THREAD_COUNT, "filesystems-test");
-
-        finishPhase.await();
-
-        for (int i = 0; i < THREAD_COUNT; i ++) {
-            cfg.set(MRJobConfig.USER_NAME, "user" + i);
-
-            Path workDir = new Path(new Path(uri), "user/user" + i);
-
-            cfg.set(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, workDir.toString());
-
-            assertEquals(workDir, FileSystem.newInstance(uri, cfg).getWorkingDirectory());
-
-            assertEquals(workDir, newUserInitWorkDir[i]);
-
-            assertEquals(new Path(new Path(uri), "user/user" + i + "/folder" + i), newWorkDir[i]);
-
-            assertEquals(new Path("/folder" + i), newAbsWorkDir[i]);
-
-            assertEquals(new Path(new Path(uri), "user/" + System.getProperty("user.name")), newInstanceWorkDir[i]);
-        }
-
-        System.out.println(System.getProperty("user.dir"));
-    }
-
-    /**
-     * Test IGFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testIgfs() throws Exception {
-        testFileSystem(URI.create(igfsScheme()));
-    }
-
-    /**
-     * Test HDFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testHdfs() throws Exception {
-        testFileSystem(URI.create("hdfs://localhost/"));
-    }
-
-    /**
-     * Test LocalFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testLocal() throws Exception {
-        testFileSystem(URI.create("file:///"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 49099fc..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java
+++ /dev/null
@@ -1,286 +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.mapreduce.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Grouping test.
- */
-public class GridHadoopGroupingTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static final String PATH_OUTPUT = "/test-out";
-
-    /** */
-    private static final GridConcurrentHashSet<UUID> vals = GridHadoopSharedMap.map(GridHadoopGroupingTest.class)
-        .put("vals", new GridConcurrentHashSet<UUID>());
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-
-    /** {@inheritDoc} */
-    protected boolean igfsEnabled() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGroupingReducer() throws Exception {
-        doTestGrouping(false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGroupingCombiner() throws Exception {
-        doTestGrouping(true);
-    }
-
-    /**
-     * @param combiner With combiner.
-     * @throws Exception If failed.
-     */
-    public void doTestGrouping(boolean combiner) throws Exception {
-        vals.clear();
-
-        Job job = Job.getInstance();
-
-        job.setInputFormatClass(InFormat.class);
-        job.setOutputFormatClass(OutFormat.class);
-
-        job.setOutputKeyClass(YearTemperature.class);
-        job.setOutputValueClass(Text.class);
-
-        job.setMapperClass(Mapper.class);
-
-        if (combiner) {
-            job.setCombinerClass(MyReducer.class);
-            job.setNumReduceTasks(0);
-            job.setCombinerKeyGroupingComparatorClass(YearComparator.class);
-        }
-        else {
-            job.setReducerClass(MyReducer.class);
-            job.setNumReduceTasks(4);
-            job.setGroupingComparatorClass(YearComparator.class);
-        }
-
-        grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 2),
-            createJobInfo(job.getConfiguration())).get(30000);
-
-        assertTrue(vals.isEmpty());
-    }
-
-    public static class MyReducer extends Reducer<YearTemperature, Text, Text, Object> {
-        /** */
-        int lastYear;
-
-        @Override protected void reduce(YearTemperature key, Iterable<Text> vals0, Context context)
-            throws IOException, InterruptedException {
-            X.println("___ : " + context.getTaskAttemptID() + " --> " + key);
-
-            Set<UUID> ids = new HashSet<>();
-
-            for (Text val : vals0)
-                assertTrue(ids.add(UUID.fromString(val.toString())));
-
-            for (Text val : vals0)
-                assertTrue(ids.remove(UUID.fromString(val.toString())));
-
-            assertTrue(ids.isEmpty());
-
-            assertTrue(key.year > lastYear);
-
-            lastYear = key.year;
-
-            for (Text val : vals0)
-                assertTrue(vals.remove(UUID.fromString(val.toString())));
-        }
-    }
-
-    public static class YearComparator implements RawComparator<YearTemperature> { // Grouping comparator.
-        /** {@inheritDoc} */
-        @Override public int compare(YearTemperature o1, YearTemperature o2) {
-            return Integer.compare(o1.year, o2.year);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-            throw new IllegalStateException();
-        }
-    }
-
-    public static class YearTemperature implements WritableComparable<YearTemperature>, Cloneable {
-        /** */
-        private int year;
-
-        /** */
-        private int temperature;
-
-        /** {@inheritDoc} */
-        @Override public void write(DataOutput out) throws IOException {
-            out.writeInt(year);
-            out.writeInt(temperature);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readFields(DataInput in) throws IOException {
-            year = in.readInt();
-            temperature = in.readInt();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            throw new IllegalStateException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() { // To be partitioned by year.
-            return year;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(YearTemperature o) {
-            int res = Integer.compare(year, o.year);
-
-            if (res != 0)
-                return res;
-
-            // Sort comparator by year and temperature, to find max for year.
-            return Integer.compare(o.temperature, temperature);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(YearTemperature.class, this);
-        }
-    }
-
-    public static class InFormat extends InputFormat<YearTemperature, Text> {
-        /** {@inheritDoc} */
-        @Override public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
-            ArrayList<InputSplit> list = new ArrayList<>();
-
-            for (int i = 0; i < 10; i++)
-                list.add(new GridHadoopSortingTest.FakeSplit(20));
-
-            return list;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RecordReader<YearTemperature, Text> createRecordReader(final InputSplit split,
-            TaskAttemptContext context) throws IOException, InterruptedException {
-            return new RecordReader<YearTemperature, Text>() {
-                /** */
-                int cnt;
-
-                /** */
-                Random rnd = new GridRandom();
-
-                /** */
-                YearTemperature key = new YearTemperature();
-
-                /** */
-                Text val = new Text();
-
-                @Override public void initialize(InputSplit split, TaskAttemptContext context) {
-                    // No-op.
-                }
-
-                @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-                    return cnt++ < split.getLength();
-                }
-
-                @Override public YearTemperature getCurrentKey() {
-                    key.year = 1990 + rnd.nextInt(10);
-                    key.temperature = 10 + rnd.nextInt(20);
-
-                    return key;
-                }
-
-                @Override public Text getCurrentValue() {
-                    UUID id = UUID.randomUUID();
-
-                    assertTrue(vals.add(id));
-
-                    val.set(id.toString());
-
-                    return val;
-                }
-
-                @Override public float getProgress() {
-                    return 0;
-                }
-
-                @Override public void close() {
-                    // No-op.
-                }
-            };
-        }
-    }
-
-    /**
-     *
-     */
-    public static class OutFormat extends OutputFormat {
-        /** {@inheritDoc} */
-        @Override public RecordWriter getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException {
-            return null;
-        }
-    }
-}


[46/58] [abbrv] incubator-ignite git commit: [IGNITE-349]: corrected license header

Posted by yz...@apache.org.
[IGNITE-349]: corrected license header


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

Branch: refs/heads/ignite-313
Commit: b83700ebab3035c3ab1f3285cb0a518e403a37cd
Parents: 6423cf0
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Mar 5 10:58:51 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu Mar 5 10:58:51 2015 +0300

----------------------------------------------------------------------
 ...HadoopSecondaryFileSystemConfigurationTest.java | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b83700eb/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index 2994107..11279ca 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.*;


[50/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
index 0000000,04a96de..4b749f3
mode 000000,100644..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
@@@ -1,0 -1,960 +1,958 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.HadoopTaskState.*;
+ 
+ /**
+  * 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 HadoopProcessDescriptor nodeDesc;
+ 
+     /** Output base. */
+     private File outputBase;
+ 
+     /** Path separator. */
+     private String pathSep;
+ 
+     /** Hadoop external communication. */
+     private HadoopExternalCommunication comm;
+ 
+     /** Starting processes. */
+     private final ConcurrentMap<UUID, HadoopProcess> runningProcsByProcId = new ConcurrentHashMap8<>();
+ 
+     /** Starting processes. */
+     private final ConcurrentMap<HadoopJobId, 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 HadoopExternalCommunication(
+             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 HadoopJobMetadata 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() == HadoopJobPhase.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, HadoopProcessDescriptor>>>() {
+                     @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> 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)) {
+             HadoopJob 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 HadoopJob job, final Collection<HadoopTaskInfo> 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());
+ 
+             HadoopTaskType taskType = F.first(tasks).type();
+ 
+             if (taskType == HadoopTaskType.SETUP || taskType == HadoopTaskType.ABORT ||
+                 taskType == HadoopTaskType.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, HadoopProcessDescriptor>>>() {
+                 @Override public void apply(
+                     IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> 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(HadoopJobId 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, HadoopJob job, Collection<HadoopTaskInfo> 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;
+             }
+ 
+             HadoopTaskExecutionRequest req = new HadoopTaskExecutionRequest();
+ 
+             req.jobId(job.id());
+             req.jobInfo(job.info());
+             req.tasks(tasks);
+ 
+             comm.sendMessage(proc.descriptor(), req);
+         }
+         finally {
+             proc.unlock();
+         }
+     }
+ 
+     /**
+      * @return External task metadata.
+      */
+     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",
+             "-DIGNITE_HOME=" + U.getIgniteHome()));
+ 
+         return meta;
+     }
+ 
+     /**
+      * @param tasks Tasks to notify about.
+      * @param state Fail state.
+      * @param e Optional error.
+      */
+     private void notifyTasksFailed(Iterable<HadoopTaskInfo> tasks, HadoopTaskState state, Throwable e) {
+         HadoopTaskStatus fail = new HadoopTaskStatus(state, e);
+ 
+         for (HadoopTaskInfo 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 HadoopJob job, final HadoopMapReducePlan plan) {
+         final UUID childProcId = UUID.randomUUID();
+ 
+         HadoopJobId jobId = job.id();
+ 
+         final HadoopProcessFuture fut = new HadoopProcessFuture(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 {
+                     HadoopExternalTaskMetadata 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, HadoopProcessDescriptor>>>() {
+             @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> 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, HadoopExternalTaskMetadata startMeta,
+         HadoopJob 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(HadoopExternalProcessStarter.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(HadoopJobId 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, HadoopJobMetadata meta) {
+         Map<Integer, HadoopProcessDescriptor> rdcAddrs = meta.reducersAddresses();
+ 
+         int rdcNum = meta.mapReducePlan().reducers();
+ 
+         HadoopProcessDescriptor[] addrs = null;
+ 
+         if (rdcAddrs != null && rdcAddrs.size() == rdcNum) {
+             addrs = new HadoopProcessDescriptor[rdcNum];
+ 
+             for (int i = 0; i < rdcNum; i++) {
+                 HadoopProcessDescriptor desc = rdcAddrs.get(i);
+ 
+                 assert desc != null : "Missing reducing address [meta=" + meta + ", rdc=" + i + ']';
+ 
+                 addrs[i] = desc;
+             }
+         }
+ 
+         try {
+             comm.sendMessage(proc.descriptor(), new HadoopJobInfoUpdateRequest(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, HadoopJob job, HadoopMapReducePlan plan) {
+         try {
+             comm.sendMessage(proc.descriptor(), new HadoopPrepareForJobRequest(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(HadoopProcessDescriptor desc, HadoopTaskFinishedMessage taskMsg) {
+         HadoopProcess proc = runningProcsByProcId.get(desc.processId());
+ 
+         if (proc != null)
+             proc.removeTask(taskMsg.taskInfo());
+ 
+         jobTracker.onTaskFinished(taskMsg.taskInfo(), taskMsg.status());
+     }
+ 
+     /**
+      *
+      */
+     private class MessageListener implements HadoopMessageListener {
+         /** {@inheritDoc} */
+         @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
+             if (!busyLock.tryReadLock())
+                 return;
+ 
+             try {
+                 if (msg instanceof HadoopProcessStartedAck) {
+                     HadoopProcess proc = runningProcsByProcId.get(desc.processId());
+ 
+                     assert proc != null : "Missing child process for processId: " + desc;
+ 
+                     HadoopProcessFuture 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 HadoopTaskFinishedMessage) {
+                     HadoopTaskFinishedMessage taskMsg = (HadoopTaskFinishedMessage)msg;
+ 
+                     processTaskFinishedMessage(desc, taskMsg);
+                 }
+                 else
+                     log.warning("Unexpected message received by node [desc=" + desc + ", msg=" + msg + ']');
+             }
+             finally {
+                 busyLock.readUnlock();
+             }
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public void onConnectionLost(HadoopProcessDescriptor 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<HadoopTaskInfo> tasks = proc.tasks();
+ 
+                     if (!F.isEmpty(tasks)) {
+                         log.warning("Lost connection with alive process (will terminate): " + desc);
+ 
+                         HadoopTaskStatus status = new HadoopTaskStatus(CRASHED,
+                             new IgniteCheckedException("Failed to run tasks (external process finished unexpectedly): " + desc));
+ 
+                         for (HadoopTaskInfo 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 HadoopJobId jobId;
+ 
+         /** Process. */
+         private Process proc;
+ 
+         /** Init future. Completes when process is ready to receive messages. */
+         private final HadoopProcessFuture initFut;
+ 
+         /** Process descriptor. */
+         private HadoopProcessDescriptor procDesc;
+ 
+         /** Reducers planned for this process. */
+         private Collection<Integer> reducers;
+ 
+         /** Tasks. */
+         private final Collection<HadoopTaskInfo> tasks = new ConcurrentLinkedDeque8<>();
+ 
+         /** Terminated flag. */
+         private volatile boolean terminated;
+ 
+         /**
+          * @param jobId Job ID.
+          * @param initFut Init future.
+          */
+         private HadoopProcess(HadoopJobId jobId, HadoopProcessFuture 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 HadoopProcessDescriptor descriptor() {
+             return procDesc;
+         }
+ 
+         /**
+          * @return Job ID.
+          */
+         public HadoopJobId jobId() {
+             return jobId;
+         }
+ 
+         /**
+          * Initialized callback.
+          *
+          * @param proc Java process representation.
+          * @param procDesc Process descriptor.
+          */
+         private void onInitialized(Process proc, HadoopProcessDescriptor 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, HadoopProcessDescriptor>>>() {
+                         @Override public void apply(
+                             IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> 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<HadoopTaskInfo> tasks) {
+             this.tasks.addAll(tasks);
+         }
+ 
+         /**
+          * Removes task when it was completed.
+          *
+          * @param task Task to remove.
+          */
+         private void removeTask(HadoopTaskInfo task) {
+             if (tasks != null)
+                 tasks.remove(task);
+         }
+ 
+         /**
+          * @return Collection of tasks.
+          */
+         private Collection<HadoopTaskInfo> 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 HadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, HadoopProcessDescriptor>> {
+         /** */
+         private static final long serialVersionUID = 0L;
+ 
+         /** Child process ID. */
+         private UUID childProcId;
+ 
+         /** Job ID. */
+         private HadoopJobId jobId;
+ 
+         /** Process descriptor. */
+         private HadoopProcessDescriptor 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 HadoopProcessFuture() {
+             // No-op.
+         }
+ 
+         /**
+          * @param ctx Kernal context.
+          */
 -        private HadoopProcessFuture(UUID childProcId, HadoopJobId jobId, GridKernalContext ctx) {
 -            super(ctx);
 -
++        private GridHadoopProcessFuture(UUID childProcId, HadoopJobId jobId, GridKernalContext 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(HadoopProcessDescriptor 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, HadoopProcessDescriptor> 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/17ac3602/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index 0000000,e95b8cb..831885f
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,440 +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.HadoopTaskType.*;
+ 
+ /**
+  * 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<>();
++    private final GridFutureAdapter<?> initFut = new GridFutureAdapter<>();
+ 
+     /** Job instance. */
+     private HadoopJob 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;
+ 
+                     HadoopTaskInfo 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 (HadoopTaskInfo 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 HadoopTaskInput createInput(HadoopTaskContext ctx)
+                                 throws IgniteCheckedException {
+                                 return shuffleJob.input(ctx);
+                             }
+ 
+                             @Override protected HadoopTaskOutput createOutput(HadoopTaskContext ctx)
+                                 throws IgniteCheckedException {
+                                 return shuffleJob.output(ctx);
+                             }
+                         });
+                     }
+                 }
+                 catch (IgniteCheckedException e) {
+                     for (HadoopTaskInfo 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) {
+         HadoopTaskInfo 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 HadoopTaskInfo 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();
+             }
+         }
+     }
+ }


[49/58] [abbrv] incubator-ignite git commit: # sprint-2 - removing @Nullable.

Posted by yz...@apache.org.
# sprint-2 - removing @Nullable.


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

Branch: refs/heads/ignite-313
Commit: e4d04761d1699f067def01a9ee8c2b7375f78418
Parents: 5d06523
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Mar 5 00:13:05 2015 -0800
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Mar 5 00:13:22 2015 -0800

----------------------------------------------------------------------
 .../org/apache/ignite/events/CacheEvent.java    | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e4d04761/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java b/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
index a14539c..276981d 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
@@ -179,7 +179,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Cache name.
      */
-    @Nullable public String cacheName() {
+    public String cacheName() {
         return cacheName;
     }
 
@@ -206,7 +206,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Node which initiated cache operation or {@code null} if that node is not available.
      */
-    @Nullable public ClusterNode eventNode() {
+    public ClusterNode eventNode() {
         return evtNode;
     }
 
@@ -216,7 +216,7 @@ public class CacheEvent extends EventAdapter {
      * @return Cache entry associated with event.
      */
     @SuppressWarnings({"unchecked"})
-    @Nullable public <K> K key() {
+    public <K> K key() {
         return (K)key;
     }
 
@@ -226,7 +226,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return ID of surrounding cache transaction.
      */
-    @Nullable public IgniteUuid xid() {
+    public IgniteUuid xid() {
         return xid;
     }
 
@@ -235,7 +235,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return ID of the lock if held.
      */
-    @Nullable public Object lockId() {
+    public Object lockId() {
         return lockId;
     }
 
@@ -245,7 +245,7 @@ public class CacheEvent extends EventAdapter {
      * @return New value associated with event (<tt>null</tt> if event is
      *      {@link EventType#EVT_CACHE_OBJECT_REMOVED}.
      */
-    @Nullable public Object newValue() {
+    public Object newValue() {
         return newVal;
     }
 
@@ -254,7 +254,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Old value associated with event.
      */
-    @Nullable public Object oldValue() {
+    public Object oldValue() {
         return oldVal;
     }
 
@@ -290,7 +290,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Subject ID.
      */
-    @Nullable public UUID subjectId() {
+    public UUID subjectId() {
         return subjId;
     }
 
@@ -299,7 +299,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Closure class name.
      */
-    @Nullable public String closureClassName() {
+    public String closureClassName() {
         return cloClsName;
     }
 
@@ -308,7 +308,7 @@ public class CacheEvent extends EventAdapter {
      *
      * @return Task name.
      */
-    @Nullable public String taskName() {
+    public String taskName() {
         return taskName;
     }
 


[54/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1


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

Branch: refs/heads/ignite-313
Commit: 6e5216292966f750b7420d072825532694a865c1
Parents: cce63fd e4d0476
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Mar 5 11:24:08 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Mar 5 11:24:08 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/events/CacheEvent.java    | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------



[14/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..d907a6c
--- /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.
+            FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+            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.setFileSystemConfiguration(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.setFileSystemConfiguration(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 FileSystemConfiguration igfsConfiguration(String gridName) throws IgniteCheckedException {
+        FileSystemConfiguration cfg = new FileSystemConfiguration();
+
+        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 IgniteHadoopIgfsSecondaryFileSystem(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++) {
+            IgniteFileSystem 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 * FileSystemConfiguration.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]);
+            }
+
+            IgniteFileSystem 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));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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..e89d015
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.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.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.igfs.secondary.*;
+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 IgfsSecondaryFileSystem secondaryFs, @Nullable Map<String, String> restCfg) throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        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.setFileSystemConfiguration(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);
+
+        IgfsSecondaryFileSystem hadoopFs = new IgniteHadoopIgfsSecondaryFileSystem(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.asSecondary(), 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/6423cf02/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/6423cf02/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);
+    }
+}


[15/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..24f10a6
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
@@ -0,0 +1,444 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.counter.HadoopCounters;
+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 HadoopTaskContext {
+    /** */
+    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 HadoopTask task;
+
+    /** Local node ID */
+    private UUID locNodeId;
+
+    /** Counters for task. */
+    private final HadoopCounters 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(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId 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 HadoopCounter> T counter(String grp, String name, Class<T> cls) {
+        return cntrs.counter(grp, name, cls);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopCounters counters() {
+        return cntrs;
+    }
+
+    /**
+     * Creates appropriate task from current task info.
+     *
+     * @return Task.
+     */
+    private HadoopTask createTask() {
+        boolean isAbort = taskInfo().type() == HadoopTaskType.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;
+
+        HadoopTask 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(HadoopTaskType 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 HadoopPartitioner 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 HadoopSerialization 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 HadoopSerialization keySerialization() throws IgniteCheckedException {
+        return getSerialization(jobCtx.getMapOutputKeyClass(), jobConf());
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopSerialization 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(HadoopInputSplit 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/6423cf02/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..3920dd5
--- /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 HadoopSerialization {
+    /** */
+    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/6423cf02/modules/hadoop/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider b/modules/hadoop/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
index fe35d5e..8d5957b 100644
--- a/modules/hadoop/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
+++ b/modules/hadoop/src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider
@@ -1 +1 @@
-org.apache.ignite.client.hadoop.GridHadoopClientProtocolProvider
+org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolEmbeddedSelfTest.java
deleted file mode 100644
index 780ce67..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolEmbeddedSelfTest.java
+++ /dev/null
@@ -1,34 +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.client.hadoop;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-
-/**
- * Hadoop client protocol tests in embedded process mode.
- */
-public class GridHadoopClientProtocolEmbeddedSelfTest extends GridHadoopClientProtocolSelfTest {
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolSelfTest.java
deleted file mode 100644
index ff8798b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolSelfTest.java
+++ /dev/null
@@ -1,633 +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.client.hadoop;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.hadoop.mapreduce.protocol.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.testframework.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Hadoop client protocol tests in external process mode.
- */
-@SuppressWarnings("ResultOfMethodCallIgnored")
-public class GridHadoopClientProtocolSelfTest extends GridHadoopAbstractSelfTest {
-    /** Input path. */
-    private static final String PATH_INPUT = "/input";
-
-    /** Output path. */
-    private static final String PATH_OUTPUT = "/output";
-
-    /** Job name. */
-    private static final String JOB_NAME = "myJob";
-
-    /** Setup lock file. */
-    private static File setupLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
-        "ignite-lock-setup.file");
-
-    /** Map lock file. */
-    private static File mapLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
-        "ignite-lock-map.file");
-
-    /** Reduce lock file. */
-    private static File reduceLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
-        "ignite-lock-reduce.file");
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 2;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean restEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrids(gridCount());
-
-        setupLockFile.delete();
-        mapLockFile.delete();
-        reduceLockFile.delete();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-
-        super.afterTestsStopped();
-
-//        GridHadoopClientProtocolProvider.cliMap.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        setupLockFile.createNewFile();
-        mapLockFile.createNewFile();
-        reduceLockFile.createNewFile();
-
-        setupLockFile.deleteOnExit();
-        mapLockFile.deleteOnExit();
-        reduceLockFile.deleteOnExit();
-
-        super.beforeTest();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        grid(0).fileSystem(GridHadoopAbstractSelfTest.igfsName).format();
-
-        setupLockFile.delete();
-        mapLockFile.delete();
-        reduceLockFile.delete();
-
-        super.afterTest();
-    }
-
-    /**
-     * Test next job ID generation.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private void tstNextJobId() throws Exception {
-        GridHadoopClientProtocolProvider provider = provider();
-
-        ClientProtocol proto = provider.create(config(GridHadoopAbstractSelfTest.REST_PORT));
-
-        JobID jobId = proto.getNewJobID();
-
-        assert jobId != null;
-        assert jobId.getJtIdentifier() != null;
-
-        JobID nextJobId = proto.getNewJobID();
-
-        assert nextJobId != null;
-        assert nextJobId.getJtIdentifier() != null;
-
-        assert !F.eq(jobId, nextJobId);
-    }
-
-    /**
-     * Tests job counters retrieval.
-     *
-     * @throws Exception If failed.
-     */
-    public void testJobCounters() throws Exception {
-        IgniteFs igfs = grid(0).fileSystem(GridHadoopAbstractSelfTest.igfsName);
-
-        igfs.mkdirs(new IgfsPath(PATH_INPUT));
-
-        try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(igfs.create(
-            new IgfsPath(PATH_INPUT + "/test.file"), true)))) {
-
-            bw.write(
-                "alpha\n" +
-                "beta\n" +
-                "gamma\n" +
-                "alpha\n" +
-                "beta\n" +
-                "gamma\n" +
-                "alpha\n" +
-                "beta\n" +
-                "gamma\n"
-            );
-        }
-
-        Configuration conf = config(GridHadoopAbstractSelfTest.REST_PORT);
-
-        final Job job = Job.getInstance(conf);
-
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(TestCountingMapper.class);
-        job.setReducerClass(TestCountingReducer.class);
-        job.setCombinerClass(TestCountingCombiner.class);
-
-        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
-        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
-
-        job.submit();
-
-        final Counter cntr = job.getCounters().findCounter(TestCounter.COUNTER1);
-
-        assertEquals(0, cntr.getValue());
-
-        cntr.increment(10);
-
-        assertEquals(10, cntr.getValue());
-
-        // Transferring to map phase.
-        setupLockFile.delete();
-
-        // Transferring to reduce phase.
-        mapLockFile.delete();
-
-        job.waitForCompletion(false);
-
-        assertEquals("job must end successfully", JobStatus.State.SUCCEEDED, job.getStatus().getState());
-
-        final Counters counters = job.getCounters();
-
-        assertNotNull("counters cannot be null", counters);
-        assertEquals("wrong counters count", 3, counters.countCounters());
-        assertEquals("wrong counter value", 15, counters.findCounter(TestCounter.COUNTER1).getValue());
-        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER2).getValue());
-        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER3).getValue());
-    }
-
-    /**
-     * Tests job counters retrieval for unknown job id.
-     *
-     * @throws Exception If failed.
-     */
-    private void tstUnknownJobCounters() throws Exception {
-        GridHadoopClientProtocolProvider provider = provider();
-
-        ClientProtocol proto = provider.create(config(GridHadoopAbstractSelfTest.REST_PORT));
-
-        try {
-            proto.getJobCounters(new JobID(UUID.randomUUID().toString(), -1));
-            fail("exception must be thrown");
-        }
-        catch (Exception e) {
-            assert e instanceof IOException : "wrong error has been thrown";
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void tstJobSubmitMap() throws Exception {
-        checkJobSubmit(true, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void tstJobSubmitMapCombine() throws Exception {
-        checkJobSubmit(false, true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void tstJobSubmitMapReduce() throws Exception {
-        checkJobSubmit(true, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void tstJobSubmitMapCombineReduce() throws Exception {
-        checkJobSubmit(false, false);
-    }
-
-    /**
-     * Test job submission.
-     *
-     * @param noCombiners Whether there are no combiners.
-     * @param noReducers Whether there are no reducers.
-     * @throws Exception If failed.
-     */
-    public void checkJobSubmit(boolean noCombiners, boolean noReducers) throws Exception {
-        IgniteFs igfs = grid(0).fileSystem(GridHadoopAbstractSelfTest.igfsName);
-
-        igfs.mkdirs(new IgfsPath(PATH_INPUT));
-
-        try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(igfs.create(
-            new IgfsPath(PATH_INPUT + "/test.file"), true)))) {
-
-            bw.write("word");
-        }
-
-        Configuration conf = config(GridHadoopAbstractSelfTest.REST_PORT);
-
-        final Job job = Job.getInstance(conf);
-
-        job.setJobName(JOB_NAME);
-
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(TestMapper.class);
-        job.setReducerClass(TestReducer.class);
-
-        if (!noCombiners)
-            job.setCombinerClass(TestCombiner.class);
-
-        if (noReducers)
-            job.setNumReduceTasks(0);
-
-        job.setInputFormatClass(TextInputFormat.class);
-        job.setOutputFormatClass(TestOutputFormat.class);
-
-        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
-        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
-
-        job.submit();
-
-        JobID jobId = job.getJobID();
-
-        // Setup phase.
-        JobStatus jobStatus = job.getStatus();
-        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-        assert jobStatus.getSetupProgress() >= 0.0f && jobStatus.getSetupProgress() < 1.0f;
-        assert jobStatus.getMapProgress() == 0.0f;
-        assert jobStatus.getReduceProgress() == 0.0f;
-
-        U.sleep(2100);
-
-        JobStatus recentJobStatus = job.getStatus();
-
-        assert recentJobStatus.getSetupProgress() > jobStatus.getSetupProgress() :
-            "Old=" + jobStatus.getSetupProgress() + ", new=" + recentJobStatus.getSetupProgress();
-
-        // Transferring to map phase.
-        setupLockFile.delete();
-
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                try {
-                    return F.eq(1.0f, job.getStatus().getSetupProgress());
-                }
-                catch (Exception e) {
-                    throw new RuntimeException("Unexpected exception.", e);
-                }
-            }
-        }, 5000L);
-
-        // Map phase.
-        jobStatus = job.getStatus();
-        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-        assert jobStatus.getSetupProgress() == 1.0f;
-        assert jobStatus.getMapProgress() >= 0.0f && jobStatus.getMapProgress() < 1.0f;
-        assert jobStatus.getReduceProgress() == 0.0f;
-
-        U.sleep(2100);
-
-        recentJobStatus = job.getStatus();
-
-        assert recentJobStatus.getMapProgress() > jobStatus.getMapProgress() :
-            "Old=" + jobStatus.getMapProgress() + ", new=" + recentJobStatus.getMapProgress();
-
-        // Transferring to reduce phase.
-        mapLockFile.delete();
-
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                try {
-                    return F.eq(1.0f, job.getStatus().getMapProgress());
-                }
-                catch (Exception e) {
-                    throw new RuntimeException("Unexpected exception.", e);
-                }
-            }
-        }, 5000L);
-
-        if (!noReducers) {
-            // Reduce phase.
-            jobStatus = job.getStatus();
-            checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-            assert jobStatus.getSetupProgress() == 1.0f;
-            assert jobStatus.getMapProgress() == 1.0f;
-            assert jobStatus.getReduceProgress() >= 0.0f && jobStatus.getReduceProgress() < 1.0f;
-
-            // Ensure that reduces progress increases.
-            U.sleep(2100);
-
-            recentJobStatus = job.getStatus();
-
-            assert recentJobStatus.getReduceProgress() > jobStatus.getReduceProgress() :
-                "Old=" + jobStatus.getReduceProgress() + ", new=" + recentJobStatus.getReduceProgress();
-
-            reduceLockFile.delete();
-        }
-
-        job.waitForCompletion(false);
-
-        jobStatus = job.getStatus();
-        checkJobStatus(job.getStatus(), jobId, JOB_NAME, JobStatus.State.SUCCEEDED, 1.0f);
-        assert jobStatus.getSetupProgress() == 1.0f;
-        assert jobStatus.getMapProgress() == 1.0f;
-        assert jobStatus.getReduceProgress() == 1.0f;
-
-        dumpIgfs(igfs, new IgfsPath(PATH_OUTPUT));
-    }
-
-    /**
-     * Dump IGFS content.
-     *
-     * @param igfs IGFS.
-     * @param path Path.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private static void dumpIgfs(IgniteFs igfs, IgfsPath path) throws Exception {
-        IgfsFile file = igfs.info(path);
-
-        assert file != null;
-
-        System.out.println(file.path());
-
-        if (file.isDirectory()) {
-            for (IgfsPath child : igfs.listPaths(path))
-                dumpIgfs(igfs, child);
-        }
-        else {
-            try (BufferedReader br = new BufferedReader(new InputStreamReader(igfs.open(path)))) {
-                String line = br.readLine();
-
-                while (line != null) {
-                    System.out.println(line);
-
-                    line = br.readLine();
-                }
-            }
-        }
-    }
-
-    /**
-     * Check job status.
-     *
-     * @param status Job status.
-     * @param expJobId Expected job ID.
-     * @param expJobName Expected job name.
-     * @param expState Expected state.
-     * @param expCleanupProgress Expected cleanup progress.
-     * @throws Exception If failed.
-     */
-    private static void checkJobStatus(JobStatus status, JobID expJobId, String expJobName,
-        JobStatus.State expState, float expCleanupProgress) throws Exception {
-        assert F.eq(status.getJobID(), expJobId) : "Expected=" + expJobId + ", actual=" + status.getJobID();
-        assert F.eq(status.getJobName(), expJobName) : "Expected=" + expJobName + ", actual=" + status.getJobName();
-        assert F.eq(status.getState(), expState) : "Expected=" + expState + ", actual=" + status.getState();
-        assert F.eq(status.getCleanupProgress(), expCleanupProgress) :
-            "Expected=" + expCleanupProgress + ", actual=" + status.getCleanupProgress();
-    }
-
-    /**
-     * @return Configuration.
-     */
-    private Configuration config(int port) {
-        Configuration conf = new Configuration();
-
-        setupFileSystems(conf);
-
-        conf.set(MRConfig.FRAMEWORK_NAME, GridHadoopClientProtocol.FRAMEWORK_NAME);
-        conf.set(MRConfig.MASTER_ADDRESS, "127.0.0.1:" + port);
-
-        conf.set("fs.defaultFS", "igfs://:" + getTestGridName(0) + "@/");
-
-        return conf;
-    }
-
-    /**
-     * @return Protocol provider.
-     */
-    private GridHadoopClientProtocolProvider provider() {
-        return new GridHadoopClientProtocolProvider();
-    }
-
-    /**
-     * Test mapper.
-     */
-    public static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
-        /** Writable container for writing word. */
-        private Text word = new Text();
-
-        /** Writable integer constant of '1' is writing as count of found words. */
-        private static final IntWritable one = new IntWritable(1);
-
-        /** {@inheritDoc} */
-        @Override public void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            while (mapLockFile.exists())
-                Thread.sleep(50);
-
-            StringTokenizer wordList = new StringTokenizer(val.toString());
-
-            while (wordList.hasMoreTokens()) {
-                word.set(wordList.nextToken());
-
-                ctx.write(word, one);
-            }
-        }
-    }
-
-    /**
-     * Test Hadoop counters.
-     */
-    public enum TestCounter {
-        COUNTER1, COUNTER2, COUNTER3
-    }
-
-    /**
-     * Test mapper that uses counters.
-     */
-    public static class TestCountingMapper extends TestMapper {
-        /** {@inheritDoc} */
-        @Override public void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            super.map(key, val, ctx);
-            ctx.getCounter(TestCounter.COUNTER1).increment(1);
-        }
-    }
-
-    /**
-     * Test combiner that counts invocations.
-     */
-    public static class TestCountingCombiner extends TestReducer {
-        @Override public void reduce(Text key, Iterable<IntWritable> values,
-            Context ctx) throws IOException, InterruptedException {
-            ctx.getCounter(TestCounter.COUNTER1).increment(1);
-            ctx.getCounter(TestCounter.COUNTER2).increment(1);
-
-            int sum = 0;
-            for (IntWritable value : values) {
-                sum += value.get();
-            }
-
-            ctx.write(key, new IntWritable(sum));
-        }
-    }
-
-    /**
-     * Test reducer that counts invocations.
-     */
-    public static class TestCountingReducer extends TestReducer {
-        @Override public void reduce(Text key, Iterable<IntWritable> values,
-            Context ctx) throws IOException, InterruptedException {
-            ctx.getCounter(TestCounter.COUNTER1).increment(1);
-            ctx.getCounter(TestCounter.COUNTER3).increment(1);
-        }
-    }
-
-    /**
-     * Test combiner.
-     */
-    public static class TestCombiner extends Reducer<Text, IntWritable, Text, IntWritable> {
-        // No-op.
-    }
-
-    public static class TestOutputFormat<K, V> extends TextOutputFormat<K, V> {
-        /** {@inheritDoc} */
-        @Override public synchronized OutputCommitter getOutputCommitter(TaskAttemptContext ctx)
-            throws IOException {
-            return new TestOutputCommitter(ctx, (FileOutputCommitter)super.getOutputCommitter(ctx));
-        }
-    }
-
-    /**
-     * Test output committer.
-     */
-    private static class TestOutputCommitter extends FileOutputCommitter {
-        /** Delegate. */
-        private final FileOutputCommitter delegate;
-
-        /**
-         * Constructor.
-         *
-         * @param ctx Task attempt context.
-         * @param delegate Delegate.
-         * @throws IOException If failed.
-         */
-        private TestOutputCommitter(TaskAttemptContext ctx, FileOutputCommitter delegate) throws IOException {
-            super(FileOutputFormat.getOutputPath(ctx), ctx);
-
-            this.delegate = delegate;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setupJob(JobContext jobCtx) throws IOException {
-            try {
-                while (setupLockFile.exists())
-                    Thread.sleep(50);
-            }
-            catch (InterruptedException ignored) {
-                throw new IOException("Interrupted.");
-            }
-
-            delegate.setupJob(jobCtx);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setupTask(TaskAttemptContext taskCtx) throws IOException {
-            delegate.setupTask(taskCtx);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean needsTaskCommit(TaskAttemptContext taskCtx) throws IOException {
-            return delegate.needsTaskCommit(taskCtx);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void commitTask(TaskAttemptContext taskCtx) throws IOException {
-            delegate.commitTask(taskCtx);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void abortTask(TaskAttemptContext taskCtx) throws IOException {
-            delegate.abortTask(taskCtx);
-        }
-    }
-
-    /**
-     * Test reducer.
-     */
-    public static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
-        /** Writable container for writing sum of word counts. */
-        private IntWritable totalWordCnt = new IntWritable();
-
-        /** {@inheritDoc} */
-        @Override public void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
-            InterruptedException {
-            while (reduceLockFile.exists())
-                Thread.sleep(50);
-
-            int wordCnt = 0;
-
-            for (IntWritable value : values)
-                wordCnt += value.get();
-
-            totalWordCnt.set(wordCnt);
-
-            ctx.write(key, totalWordCnt);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java
new file mode 100644
index 0000000..ffa20d1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.client.hadoop;
+
+import org.apache.ignite.configuration.*;
+
+/**
+ * Hadoop client protocol tests in embedded process mode.
+ */
+public class HadoopClientProtocolEmbeddedSelfTest extends HadoopClientProtocolSelfTest {
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
new file mode 100644
index 0000000..d19a8ea
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
@@ -0,0 +1,635 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.client.hadoop;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.mapreduce.protocol.*;
+import org.apache.ignite.*;
+import org.apache.ignite.hadoop.mapreduce.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.proto.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Hadoop client protocol tests in external process mode.
+ */
+@SuppressWarnings("ResultOfMethodCallIgnored")
+public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
+    /** Input path. */
+    private static final String PATH_INPUT = "/input";
+
+    /** Output path. */
+    private static final String PATH_OUTPUT = "/output";
+
+    /** Job name. */
+    private static final String JOB_NAME = "myJob";
+
+    /** Setup lock file. */
+    private static File setupLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
+        "ignite-lock-setup.file");
+
+    /** Map lock file. */
+    private static File mapLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
+        "ignite-lock-map.file");
+
+    /** Reduce lock file. */
+    private static File reduceLockFile = new File(U.isWindows() ? System.getProperty("java.io.tmpdir") : "/tmp",
+        "ignite-lock-reduce.file");
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean restEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(gridCount());
+
+        setupLockFile.delete();
+        mapLockFile.delete();
+        reduceLockFile.delete();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+
+//        IgniteHadoopClientProtocolProvider.cliMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        setupLockFile.createNewFile();
+        mapLockFile.createNewFile();
+        reduceLockFile.createNewFile();
+
+        setupLockFile.deleteOnExit();
+        mapLockFile.deleteOnExit();
+        reduceLockFile.deleteOnExit();
+
+        super.beforeTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        grid(0).fileSystem(HadoopAbstractSelfTest.igfsName).format();
+
+        setupLockFile.delete();
+        mapLockFile.delete();
+        reduceLockFile.delete();
+
+        super.afterTest();
+    }
+
+    /**
+     * Test next job ID generation.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private void tstNextJobId() throws Exception {
+        IgniteHadoopClientProtocolProvider provider = provider();
+
+        ClientProtocol proto = provider.create(config(HadoopAbstractSelfTest.REST_PORT));
+
+        JobID jobId = proto.getNewJobID();
+
+        assert jobId != null;
+        assert jobId.getJtIdentifier() != null;
+
+        JobID nextJobId = proto.getNewJobID();
+
+        assert nextJobId != null;
+        assert nextJobId.getJtIdentifier() != null;
+
+        assert !F.eq(jobId, nextJobId);
+    }
+
+    /**
+     * Tests job counters retrieval.
+     *
+     * @throws Exception If failed.
+     */
+    public void testJobCounters() throws Exception {
+        IgniteFileSystem igfs = grid(0).fileSystem(HadoopAbstractSelfTest.igfsName);
+
+        igfs.mkdirs(new IgfsPath(PATH_INPUT));
+
+        try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(igfs.create(
+            new IgfsPath(PATH_INPUT + "/test.file"), true)))) {
+
+            bw.write(
+                "alpha\n" +
+                "beta\n" +
+                "gamma\n" +
+                "alpha\n" +
+                "beta\n" +
+                "gamma\n" +
+                "alpha\n" +
+                "beta\n" +
+                "gamma\n"
+            );
+        }
+
+        Configuration conf = config(HadoopAbstractSelfTest.REST_PORT);
+
+        final Job job = Job.getInstance(conf);
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(TestCountingMapper.class);
+        job.setReducerClass(TestCountingReducer.class);
+        job.setCombinerClass(TestCountingCombiner.class);
+
+        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
+        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
+
+        job.submit();
+
+        final Counter cntr = job.getCounters().findCounter(TestCounter.COUNTER1);
+
+        assertEquals(0, cntr.getValue());
+
+        cntr.increment(10);
+
+        assertEquals(10, cntr.getValue());
+
+        // Transferring to map phase.
+        setupLockFile.delete();
+
+        // Transferring to reduce phase.
+        mapLockFile.delete();
+
+        job.waitForCompletion(false);
+
+        assertEquals("job must end successfully", JobStatus.State.SUCCEEDED, job.getStatus().getState());
+
+        final Counters counters = job.getCounters();
+
+        assertNotNull("counters cannot be null", counters);
+        assertEquals("wrong counters count", 3, counters.countCounters());
+        assertEquals("wrong counter value", 15, counters.findCounter(TestCounter.COUNTER1).getValue());
+        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER2).getValue());
+        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER3).getValue());
+    }
+
+    /**
+     * Tests job counters retrieval for unknown job id.
+     *
+     * @throws Exception If failed.
+     */
+    private void tstUnknownJobCounters() throws Exception {
+        IgniteHadoopClientProtocolProvider provider = provider();
+
+        ClientProtocol proto = provider.create(config(HadoopAbstractSelfTest.REST_PORT));
+
+        try {
+            proto.getJobCounters(new JobID(UUID.randomUUID().toString(), -1));
+            fail("exception must be thrown");
+        }
+        catch (Exception e) {
+            assert e instanceof IOException : "wrong error has been thrown";
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void tstJobSubmitMap() throws Exception {
+        checkJobSubmit(true, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void tstJobSubmitMapCombine() throws Exception {
+        checkJobSubmit(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void tstJobSubmitMapReduce() throws Exception {
+        checkJobSubmit(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void tstJobSubmitMapCombineReduce() throws Exception {
+        checkJobSubmit(false, false);
+    }
+
+    /**
+     * Test job submission.
+     *
+     * @param noCombiners Whether there are no combiners.
+     * @param noReducers Whether there are no reducers.
+     * @throws Exception If failed.
+     */
+    public void checkJobSubmit(boolean noCombiners, boolean noReducers) throws Exception {
+        IgniteFileSystem igfs = grid(0).fileSystem(HadoopAbstractSelfTest.igfsName);
+
+        igfs.mkdirs(new IgfsPath(PATH_INPUT));
+
+        try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(igfs.create(
+            new IgfsPath(PATH_INPUT + "/test.file"), true)))) {
+
+            bw.write("word");
+        }
+
+        Configuration conf = config(HadoopAbstractSelfTest.REST_PORT);
+
+        final Job job = Job.getInstance(conf);
+
+        job.setJobName(JOB_NAME);
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(IntWritable.class);
+
+        job.setMapperClass(TestMapper.class);
+        job.setReducerClass(TestReducer.class);
+
+        if (!noCombiners)
+            job.setCombinerClass(TestCombiner.class);
+
+        if (noReducers)
+            job.setNumReduceTasks(0);
+
+        job.setInputFormatClass(TextInputFormat.class);
+        job.setOutputFormatClass(TestOutputFormat.class);
+
+        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
+        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
+
+        job.submit();
+
+        JobID jobId = job.getJobID();
+
+        // Setup phase.
+        JobStatus jobStatus = job.getStatus();
+        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+        assert jobStatus.getSetupProgress() >= 0.0f && jobStatus.getSetupProgress() < 1.0f;
+        assert jobStatus.getMapProgress() == 0.0f;
+        assert jobStatus.getReduceProgress() == 0.0f;
+
+        U.sleep(2100);
+
+        JobStatus recentJobStatus = job.getStatus();
+
+        assert recentJobStatus.getSetupProgress() > jobStatus.getSetupProgress() :
+            "Old=" + jobStatus.getSetupProgress() + ", new=" + recentJobStatus.getSetupProgress();
+
+        // Transferring to map phase.
+        setupLockFile.delete();
+
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return F.eq(1.0f, job.getStatus().getSetupProgress());
+                }
+                catch (Exception e) {
+                    throw new RuntimeException("Unexpected exception.", e);
+                }
+            }
+        }, 5000L);
+
+        // Map phase.
+        jobStatus = job.getStatus();
+        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+        assert jobStatus.getSetupProgress() == 1.0f;
+        assert jobStatus.getMapProgress() >= 0.0f && jobStatus.getMapProgress() < 1.0f;
+        assert jobStatus.getReduceProgress() == 0.0f;
+
+        U.sleep(2100);
+
+        recentJobStatus = job.getStatus();
+
+        assert recentJobStatus.getMapProgress() > jobStatus.getMapProgress() :
+            "Old=" + jobStatus.getMapProgress() + ", new=" + recentJobStatus.getMapProgress();
+
+        // Transferring to reduce phase.
+        mapLockFile.delete();
+
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return F.eq(1.0f, job.getStatus().getMapProgress());
+                }
+                catch (Exception e) {
+                    throw new RuntimeException("Unexpected exception.", e);
+                }
+            }
+        }, 5000L);
+
+        if (!noReducers) {
+            // Reduce phase.
+            jobStatus = job.getStatus();
+            checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+            assert jobStatus.getSetupProgress() == 1.0f;
+            assert jobStatus.getMapProgress() == 1.0f;
+            assert jobStatus.getReduceProgress() >= 0.0f && jobStatus.getReduceProgress() < 1.0f;
+
+            // Ensure that reduces progress increases.
+            U.sleep(2100);
+
+            recentJobStatus = job.getStatus();
+
+            assert recentJobStatus.getReduceProgress() > jobStatus.getReduceProgress() :
+                "Old=" + jobStatus.getReduceProgress() + ", new=" + recentJobStatus.getReduceProgress();
+
+            reduceLockFile.delete();
+        }
+
+        job.waitForCompletion(false);
+
+        jobStatus = job.getStatus();
+        checkJobStatus(job.getStatus(), jobId, JOB_NAME, JobStatus.State.SUCCEEDED, 1.0f);
+        assert jobStatus.getSetupProgress() == 1.0f;
+        assert jobStatus.getMapProgress() == 1.0f;
+        assert jobStatus.getReduceProgress() == 1.0f;
+
+        dumpIgfs(igfs, new IgfsPath(PATH_OUTPUT));
+    }
+
+    /**
+     * Dump IGFS content.
+     *
+     * @param igfs IGFS.
+     * @param path Path.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private static void dumpIgfs(IgniteFileSystem igfs, IgfsPath path) throws Exception {
+        IgfsFile file = igfs.info(path);
+
+        assert file != null;
+
+        System.out.println(file.path());
+
+        if (file.isDirectory()) {
+            for (IgfsPath child : igfs.listPaths(path))
+                dumpIgfs(igfs, child);
+        }
+        else {
+            try (BufferedReader br = new BufferedReader(new InputStreamReader(igfs.open(path)))) {
+                String line = br.readLine();
+
+                while (line != null) {
+                    System.out.println(line);
+
+                    line = br.readLine();
+                }
+            }
+        }
+    }
+
+    /**
+     * Check job status.
+     *
+     * @param status Job status.
+     * @param expJobId Expected job ID.
+     * @param expJobName Expected job name.
+     * @param expState Expected state.
+     * @param expCleanupProgress Expected cleanup progress.
+     * @throws Exception If failed.
+     */
+    private static void checkJobStatus(JobStatus status, JobID expJobId, String expJobName,
+        JobStatus.State expState, float expCleanupProgress) throws Exception {
+        assert F.eq(status.getJobID(), expJobId) : "Expected=" + expJobId + ", actual=" + status.getJobID();
+        assert F.eq(status.getJobName(), expJobName) : "Expected=" + expJobName + ", actual=" + status.getJobName();
+        assert F.eq(status.getState(), expState) : "Expected=" + expState + ", actual=" + status.getState();
+        assert F.eq(status.getCleanupProgress(), expCleanupProgress) :
+            "Expected=" + expCleanupProgress + ", actual=" + status.getCleanupProgress();
+    }
+
+    /**
+     * @return Configuration.
+     */
+    private Configuration config(int port) {
+        Configuration conf = new Configuration();
+
+        setupFileSystems(conf);
+
+        conf.set(MRConfig.FRAMEWORK_NAME, HadoopClientProtocol.FRAMEWORK_NAME);
+        conf.set(MRConfig.MASTER_ADDRESS, "127.0.0.1:" + port);
+
+        conf.set("fs.defaultFS", "igfs://:" + getTestGridName(0) + "@/");
+
+        return conf;
+    }
+
+    /**
+     * @return Protocol provider.
+     */
+    private IgniteHadoopClientProtocolProvider provider() {
+        return new IgniteHadoopClientProtocolProvider();
+    }
+
+    /**
+     * Test mapper.
+     */
+    public static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
+        /** Writable container for writing word. */
+        private Text word = new Text();
+
+        /** Writable integer constant of '1' is writing as count of found words. */
+        private static final IntWritable one = new IntWritable(1);
+
+        /** {@inheritDoc} */
+        @Override public void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            while (mapLockFile.exists())
+                Thread.sleep(50);
+
+            StringTokenizer wordList = new StringTokenizer(val.toString());
+
+            while (wordList.hasMoreTokens()) {
+                word.set(wordList.nextToken());
+
+                ctx.write(word, one);
+            }
+        }
+    }
+
+    /**
+     * Test Hadoop counters.
+     */
+    public enum TestCounter {
+        COUNTER1, COUNTER2, COUNTER3
+    }
+
+    /**
+     * Test mapper that uses counters.
+     */
+    public static class TestCountingMapper extends TestMapper {
+        /** {@inheritDoc} */
+        @Override public void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
+            super.map(key, val, ctx);
+            ctx.getCounter(TestCounter.COUNTER1).increment(1);
+        }
+    }
+
+    /**
+     * Test combiner that counts invocations.
+     */
+    public static class TestCountingCombiner extends TestReducer {
+        @Override public void reduce(Text key, Iterable<IntWritable> values,
+            Context ctx) throws IOException, InterruptedException {
+            ctx.getCounter(TestCounter.COUNTER1).increment(1);
+            ctx.getCounter(TestCounter.COUNTER2).increment(1);
+
+            int sum = 0;
+            for (IntWritable value : values) {
+                sum += value.get();
+            }
+
+            ctx.write(key, new IntWritable(sum));
+        }
+    }
+
+    /**
+     * Test reducer that counts invocations.
+     */
+    public static class TestCountingReducer extends TestReducer {
+        @Override public void reduce(Text key, Iterable<IntWritable> values,
+            Context ctx) throws IOException, InterruptedException {
+            ctx.getCounter(TestCounter.COUNTER1).increment(1);
+            ctx.getCounter(TestCounter.COUNTER3).increment(1);
+        }
+    }
+
+    /**
+     * Test combiner.
+     */
+    public static class TestCombiner extends Reducer<Text, IntWritable, Text, IntWritable> {
+        // No-op.
+    }
+
+    public static class TestOutputFormat<K, V> extends TextOutputFormat<K, V> {
+        /** {@inheritDoc} */
+        @Override public synchronized OutputCommitter getOutputCommitter(TaskAttemptContext ctx)
+            throws IOException {
+            return new TestOutputCommitter(ctx, (FileOutputCommitter)super.getOutputCommitter(ctx));
+        }
+    }
+
+    /**
+     * Test output committer.
+     */
+    private static class TestOutputCommitter extends FileOutputCommitter {
+        /** Delegate. */
+        private final FileOutputCommitter delegate;
+
+        /**
+         * Constructor.
+         *
+         * @param ctx Task attempt context.
+         * @param delegate Delegate.
+         * @throws IOException If failed.
+         */
+        private TestOutputCommitter(TaskAttemptContext ctx, FileOutputCommitter delegate) throws IOException {
+            super(FileOutputFormat.getOutputPath(ctx), ctx);
+
+            this.delegate = delegate;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setupJob(JobContext jobCtx) throws IOException {
+            try {
+                while (setupLockFile.exists())
+                    Thread.sleep(50);
+            }
+            catch (InterruptedException ignored) {
+                throw new IOException("Interrupted.");
+            }
+
+            delegate.setupJob(jobCtx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setupTask(TaskAttemptContext taskCtx) throws IOException {
+            delegate.setupTask(taskCtx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean needsTaskCommit(TaskAttemptContext taskCtx) throws IOException {
+            return delegate.needsTaskCommit(taskCtx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void commitTask(TaskAttemptContext taskCtx) throws IOException {
+            delegate.commitTask(taskCtx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void abortTask(TaskAttemptContext taskCtx) throws IOException {
+            delegate.abortTask(taskCtx);
+        }
+    }
+
+    /**
+     * Test reducer.
+     */
+    public static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
+        /** Writable container for writing sum of word counts. */
+        private IntWritable totalWordCnt = new IntWritable();
+
+        /** {@inheritDoc} */
+        @Override public void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
+            InterruptedException {
+            while (reduceLockFile.exists())
+                Thread.sleep(50);
+
+            int wordCnt = 0;
+
+            for (IntWritable value : values)
+                wordCnt += value.get();
+
+            totalWordCnt.set(wordCnt);
+
+            ctx.write(key, totalWordCnt);
+        }
+    }
+}


[12/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 a54e264..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.igfs.hadoop.*;
-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.igfs.hadoop.IgfsHadoopParameters.*;
-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 IgfsHadoopFileSystemWrapper(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/6423cf02/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/6423cf02/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);
-    }
-}


[24/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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..65d9268
--- /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(HadoopJobInfo 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(HadoopTaskContext 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 HadoopTaskInput input(HadoopTaskContext 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(HadoopTaskContext 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/6423cf02/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..f524bdc
--- /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(HadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) {
+        super(jobInfo, mem);
+
+        assert U.isPow2(cap) : cap;
+
+        tbl = new long[cap];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Adder startAdding(HadoopTaskContext 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(HadoopTaskContext 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/6423cf02/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..16aa673
--- /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(HadoopJobInfo jobInfo, GridUnsafeMemory mem) {
+        super(jobInfo, mem);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("visit");
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopTaskInput input(HadoopTaskContext 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(HadoopSerialization 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 HadoopTaskInput {
+        /** */
+        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(HadoopTaskContext 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/6423cf02/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..5def6d3
--- /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(HadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param taskCtx Task context.
+     * @return Task input.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopTaskInput input(HadoopTaskContext taskCtx)
+        throws IgniteCheckedException;
+
+    /** {@inheritDoc} */
+    @Override public void close();
+
+    /**
+     * Adder.
+     */
+    public interface Adder extends HadoopTaskOutput {
+        /**
+         * @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/6423cf02/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..7f332aa
--- /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.HadoopJobProperty.*;
+
+/**
+ * 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(HadoopJobInfo 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 HadoopSerialization ser;
+
+        /** */
+        private final HadoopDataInStream in = new HadoopDataInStream(mem);
+
+        /**
+         * @param ser Serialization.
+         */
+        protected ReaderBase(HadoopSerialization 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 HadoopSerialization keySer;
+
+        /** */
+        protected final HadoopSerialization 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(HadoopTaskContext 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, HadoopSerialization 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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..69aa7a7
--- /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(HadoopJobInfo 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(HadoopTaskContext ctx) throws IgniteCheckedException {
+        return new AdderImpl(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopTaskInput input(HadoopTaskContext 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(HadoopSerialization 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(HadoopTaskContext 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 HadoopTaskInput {
+        /** */
+        private long metaPtr = heads;
+
+        /** */
+        private final Reader keyReader;
+
+        /** */
+        private final Reader valReader;
+
+        /**
+         * @param taskCtx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Input(HadoopTaskContext 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 HadoopTaskInput {
+        /** */
+        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();
+        }
+    }
+}


[06/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 3aa74d0..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
+++ /dev/null
@@ -1,330 +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.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Job tracker self test.
- */
-public class GridHadoopJobTrackerSelfTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static final String PATH_OUTPUT = "/test-out";
-
-    /** Test block count parameter name. */
-    private static final int BLOCK_CNT = 10;
-
-    /** */
-    private static GridHadoopSharedMap m = GridHadoopSharedMap.map(GridHadoopJobTrackerSelfTest.class);
-
-    /** Map task execution count. */
-    private static final AtomicInteger mapExecCnt = m.put("mapExecCnt", new AtomicInteger());
-
-    /** Reduce task execution count. */
-    private static final AtomicInteger reduceExecCnt = m.put("reduceExecCnt", new AtomicInteger());
-
-    /** Reduce task execution count. */
-    private static final AtomicInteger combineExecCnt = m.put("combineExecCnt", new AtomicInteger());
-
-    /** */
-    private static final Map<String, CountDownLatch> latch = m.put("latch", new HashMap<String, CountDownLatch>());
-
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-
-        super.afterTestsStopped();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        latch.put("mapAwaitLatch", new CountDownLatch(1));
-        latch.put("reduceAwaitLatch", new CountDownLatch(1));
-        latch.put("combineAwaitLatch", new CountDownLatch(1));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        mapExecCnt.set(0);
-        combineExecCnt.set(0);
-        reduceExecCnt.set(0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setMapReducePlanner(new GridHadoopTestRoundRobinMrPlanner());
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleTaskSubmit() throws Exception {
-        try {
-            UUID globalId = UUID.randomUUID();
-
-            Job job = Job.getInstance();
-            setupFileSystems(job.getConfiguration());
-
-            job.setMapperClass(TestMapper.class);
-            job.setReducerClass(TestReducer.class);
-            job.setInputFormatClass(InFormat.class);
-
-            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT + "1"));
-
-            GridHadoopJobId jobId = new GridHadoopJobId(globalId, 1);
-
-            grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
-
-            checkStatus(jobId, false);
-
-            info("Releasing map latch.");
-
-            latch.get("mapAwaitLatch").countDown();
-
-            checkStatus(jobId, false);
-
-            info("Releasing reduce latch.");
-
-            latch.get("reduceAwaitLatch").countDown();
-
-            checkStatus(jobId, true);
-
-            assertEquals(10, mapExecCnt.get());
-            assertEquals(0, combineExecCnt.get());
-            assertEquals(1, reduceExecCnt.get());
-        }
-        finally {
-            // Safety.
-            latch.get("mapAwaitLatch").countDown();
-            latch.get("combineAwaitLatch").countDown();
-            latch.get("reduceAwaitLatch").countDown();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTaskWithCombinerPerMap() throws Exception {
-        try {
-            UUID globalId = UUID.randomUUID();
-
-            Job job = Job.getInstance();
-            setupFileSystems(job.getConfiguration());
-
-            job.setMapperClass(TestMapper.class);
-            job.setReducerClass(TestReducer.class);
-            job.setCombinerClass(TestCombiner.class);
-            job.setInputFormatClass(InFormat.class);
-
-            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT + "2"));
-
-            GridHadoopJobId jobId = new GridHadoopJobId(globalId, 1);
-
-            grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
-
-            checkStatus(jobId, false);
-
-            info("Releasing map latch.");
-
-            latch.get("mapAwaitLatch").countDown();
-
-            checkStatus(jobId, false);
-
-            // All maps are completed. We have a combiner, so no reducers should be executed
-            // before combiner latch is released.
-
-            U.sleep(50);
-
-            assertEquals(0, reduceExecCnt.get());
-
-            info("Releasing combiner latch.");
-
-            latch.get("combineAwaitLatch").countDown();
-
-            checkStatus(jobId, false);
-
-            info("Releasing reduce latch.");
-
-            latch.get("reduceAwaitLatch").countDown();
-
-            checkStatus(jobId, true);
-
-            assertEquals(10, mapExecCnt.get());
-            assertEquals(10, combineExecCnt.get());
-            assertEquals(1, reduceExecCnt.get());
-        }
-        finally {
-            // Safety.
-            latch.get("mapAwaitLatch").countDown();
-            latch.get("combineAwaitLatch").countDown();
-            latch.get("reduceAwaitLatch").countDown();
-        }
-    }
-
-    /**
-     * Checks job execution status.
-     *
-     * @param jobId Job ID.
-     * @param complete Completion status.
-     * @throws Exception If failed.
-     */
-    private void checkStatus(GridHadoopJobId jobId, boolean complete) throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            IgniteKernal kernal = (IgniteKernal)grid(i);
-
-            GridHadoop hadoop = kernal.hadoop();
-
-            GridHadoopJobStatus stat = hadoop.status(jobId);
-
-            assert stat != null;
-
-            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
-
-            if (!complete)
-                assertFalse(fut.isDone());
-            else {
-                info("Waiting for status future completion on node [idx=" + i + ", nodeId=" +
-                    kernal.getLocalNodeId() + ']');
-
-                fut.get();
-            }
-        }
-    }
-
-    /**
-     * Test input format
-     */
-    public static class InFormat extends InputFormat {
-
-        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException, InterruptedException {
-            List<InputSplit> res = new ArrayList<>(BLOCK_CNT);
-
-            for (int i = 0; i < BLOCK_CNT; i++)
-                try {
-                    res.add(new FileSplit(new Path(new URI("someFile")), i, i + 1, new String[] {"localhost"}));
-                }
-                catch (URISyntaxException e) {
-                    throw new IOException(e);
-                }
-
-            return res;
-        }
-
-        @Override public RecordReader createRecordReader(InputSplit split, TaskAttemptContext ctx) throws IOException, InterruptedException {
-            return new RecordReader() {
-                @Override public void initialize(InputSplit split, TaskAttemptContext ctx) {
-                }
-
-                @Override public boolean nextKeyValue() {
-                    return false;
-                }
-
-                @Override public Object getCurrentKey() {
-                    return null;
-                }
-
-                @Override public Object getCurrentValue() {
-                    return null;
-                }
-
-                @Override public float getProgress() {
-                    return 0;
-                }
-
-                @Override public void close() {
-
-                }
-            };
-        }
-    }
-
-    /**
-     * Test mapper.
-     */
-    private static class TestMapper extends Mapper {
-        @Override public void run(Context ctx) throws IOException, InterruptedException {
-            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
-
-            latch.get("mapAwaitLatch").await();
-
-            mapExecCnt.incrementAndGet();
-
-            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
-        }
-    }
-
-    /**
-     * Test reducer.
-     */
-    private static class TestReducer extends Reducer {
-        @Override public void run(Context ctx) throws IOException, InterruptedException {
-            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
-
-            latch.get("reduceAwaitLatch").await();
-
-            reduceExecCnt.incrementAndGet();
-
-            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
-        }
-    }
-
-    /**
-     * Test combiner.
-     */
-    private static class TestCombiner extends Reducer {
-        @Override public void run(Context ctx) throws IOException, InterruptedException {
-            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
-
-            latch.get("combineAwaitLatch").await();
-
-            combineExecCnt.incrementAndGet();
-
-            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index dda041c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
+++ /dev/null
@@ -1,245 +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.io.serializer.*;
-import org.apache.hadoop.mapred.*;
-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.igfs.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.examples.*;
-
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Tests map-reduce execution with embedded mode.
- */
-public class GridHadoopMapReduceEmbeddedSelfTest extends GridHadoopMapReduceTest {
-    /** */
-    private static Map<String, Boolean> flags = GridHadoopSharedMap.map(GridHadoopMapReduceEmbeddedSelfTest.class)
-        .put("flags", new HashMap<String, Boolean>());
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-
-    /**
-     * Tests whole job execution with all phases in old and new versions of API with definition of custom
-     * Serialization, Partitioner and IO formats.
-     * @throws Exception If fails.
-     */
-    public void testMultiReducerWholeMapReduceExecution() throws Exception {
-        IgfsPath inDir = new IgfsPath(PATH_INPUT);
-
-        igfs.mkdirs(inDir);
-
-        IgfsPath inFile = new IgfsPath(inDir, GridHadoopWordCount2.class.getSimpleName() + "-input");
-
-        generateTestFile(inFile.toString(), "key1", 10000, "key2", 20000, "key3", 15000, "key4", 7000, "key5", 12000,
-            "key6", 18000 );
-
-        for (int i = 0; i < 2; i++) {
-            boolean useNewAPI = i == 1;
-
-            igfs.delete(new IgfsPath(PATH_OUTPUT), true);
-
-            flags.put("serializationWasConfigured", false);
-            flags.put("partitionerWasConfigured", false);
-            flags.put("inputFormatWasConfigured", false);
-            flags.put("outputFormatWasConfigured", false);
-
-            JobConf jobConf = new JobConf();
-
-            jobConf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
-
-            //To split into about 6-7 items for v2
-            jobConf.setInt(FileInputFormat.SPLIT_MAXSIZE, 65000);
-
-            //For v1
-            jobConf.setInt("fs.local.block.size", 65000);
-
-            // File system coordinates.
-            setupFileSystems(jobConf);
-
-            GridHadoopWordCount1.setTasksClasses(jobConf, !useNewAPI, !useNewAPI, !useNewAPI);
-
-            if (!useNewAPI) {
-                jobConf.setPartitionerClass(CustomV1Partitioner.class);
-                jobConf.setInputFormat(CustomV1InputFormat.class);
-                jobConf.setOutputFormat(CustomV1OutputFormat.class);
-            }
-
-            Job job = Job.getInstance(jobConf);
-
-            GridHadoopWordCount2.setTasksClasses(job, useNewAPI, useNewAPI, useNewAPI);
-
-            if (useNewAPI) {
-                job.setPartitionerClass(CustomV2Partitioner.class);
-                job.setInputFormatClass(CustomV2InputFormat.class);
-                job.setOutputFormatClass(CustomV2OutputFormat.class);
-            }
-
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(IntWritable.class);
-
-            FileInputFormat.setInputPaths(job, new Path(igfsScheme() + inFile.toString()));
-            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
-
-            job.setNumReduceTasks(3);
-
-            job.setJarByClass(GridHadoopWordCount2.class);
-
-            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
-                    createJobInfo(job.getConfiguration()));
-
-            fut.get();
-
-            assertTrue("Serialization was configured (new API is " + useNewAPI + ")",
-                 flags.get("serializationWasConfigured"));
-
-            assertTrue("Partitioner was configured (new API is = " + useNewAPI + ")",
-                 flags.get("partitionerWasConfigured"));
-
-            assertTrue("Input format was configured (new API is = " + useNewAPI + ")",
-                 flags.get("inputFormatWasConfigured"));
-
-            assertTrue("Output format was configured (new API is = " + useNewAPI + ")",
-                 flags.get("outputFormatWasConfigured"));
-
-            assertEquals("Use new API = " + useNewAPI,
-                "key3\t15000\n" +
-                "key6\t18000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00000")
-            );
-
-            assertEquals("Use new API = " + useNewAPI,
-                "key1\t10000\n" +
-                "key4\t7000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00001")
-            );
-
-            assertEquals("Use new API = " + useNewAPI,
-                "key2\t20000\n" +
-                "key5\t12000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00002")
-            );
-
-        }
-    }
-
-    /**
-     * Custom serialization class that inherits behaviour of native {@link WritableSerialization}.
-     */
-    protected static class CustomSerialization extends WritableSerialization {
-        @Override public void setConf(Configuration conf) {
-            super.setConf(conf);
-
-            flags.put("serializationWasConfigured", true);
-        }
-    }
-
-    /**
-     * Custom implementation of Partitioner in v1 API.
-     */
-    private static class CustomV1Partitioner extends org.apache.hadoop.mapred.lib.HashPartitioner {
-        /** {@inheritDoc} */
-        @Override public void configure(JobConf job) {
-            flags.put("partitionerWasConfigured", true);
-        }
-    }
-
-    /**
-     * Custom implementation of Partitioner in v2 API.
-     */
-    private static class CustomV2Partitioner extends org.apache.hadoop.mapreduce.lib.partition.HashPartitioner
-            implements Configurable {
-        /** {@inheritDoc} */
-        @Override public void setConf(Configuration conf) {
-            flags.put("partitionerWasConfigured", true);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Configuration getConf() {
-            return null;
-        }
-    }
-
-    /**
-     * Custom implementation of InputFormat in v2 API.
-     */
-    private static class CustomV2InputFormat extends org.apache.hadoop.mapreduce.lib.input.TextInputFormat implements Configurable {
-        /** {@inheritDoc} */
-        @Override public void setConf(Configuration conf) {
-            flags.put("inputFormatWasConfigured", true);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Configuration getConf() {
-            return null;
-        }
-    }
-
-    /**
-     * Custom implementation of OutputFormat in v2 API.
-     */
-    private static class CustomV2OutputFormat extends org.apache.hadoop.mapreduce.lib.output.TextOutputFormat implements Configurable {
-        /** {@inheritDoc} */
-        @Override public void setConf(Configuration conf) {
-            flags.put("outputFormatWasConfigured", true);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Configuration getConf() {
-            return null;
-        }
-    }
-
-    /**
-     * Custom implementation of InputFormat in v1 API.
-     */
-    private static class CustomV1InputFormat extends org.apache.hadoop.mapred.TextInputFormat {
-        /** {@inheritDoc} */
-        @Override public void configure(JobConf job) {
-            super.configure(job);
-
-            flags.put("inputFormatWasConfigured", true);
-        }
-    }
-
-    /**
-     * Custom implementation of OutputFormat in v1 API.
-     */
-    private static class CustomV1OutputFormat extends org.apache.hadoop.mapred.TextOutputFormat implements JobConfigurable {
-        /** {@inheritDoc} */
-        @Override public void configure(JobConf job) {
-            flags.put("outputFormatWasConfigured", true);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 072e764..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
+++ /dev/null
@@ -1,195 +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.mapred.*;
-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.igfs.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.examples.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.testframework.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Test of whole cycle of map-reduce processing via Job tracker.
- */
-public class GridHadoopMapReduceTest extends GridHadoopAbstractWordCountTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-
-    /**
-     * Tests whole job execution with all phases in all combination of new and old versions of API.
-     * @throws Exception If fails.
-     */
-    public void testWholeMapReduceExecution() throws Exception {
-        IgfsPath inDir = new IgfsPath(PATH_INPUT);
-
-        igfs.mkdirs(inDir);
-
-        IgfsPath inFile = new IgfsPath(inDir, GridHadoopWordCount2.class.getSimpleName() + "-input");
-
-        generateTestFile(inFile.toString(), "red", 100000, "blue", 200000, "green", 150000, "yellow", 70000 );
-
-        for (int i = 0; i < 8; i++) {
-            igfs.delete(new IgfsPath(PATH_OUTPUT), true);
-
-            boolean useNewMapper = (i & 1) == 0;
-            boolean useNewCombiner = (i & 2) == 0;
-            boolean useNewReducer = (i & 4) == 0;
-
-            JobConf jobConf = new JobConf();
-
-            jobConf.set(JOB_COUNTER_WRITER_PROPERTY, GridHadoopFSCounterWriter.class.getName());
-            jobConf.setUser("yyy");
-            jobConf.set(GridHadoopFSCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
-
-            //To split into about 40 items for v2
-            jobConf.setInt(FileInputFormat.SPLIT_MAXSIZE, 65000);
-
-            //For v1
-            jobConf.setInt("fs.local.block.size", 65000);
-
-            // File system coordinates.
-            setupFileSystems(jobConf);
-
-            GridHadoopWordCount1.setTasksClasses(jobConf, !useNewMapper, !useNewCombiner, !useNewReducer);
-
-            Job job = Job.getInstance(jobConf);
-
-            GridHadoopWordCount2.setTasksClasses(job, useNewMapper, useNewCombiner, useNewReducer);
-
-            job.setOutputKeyClass(Text.class);
-            job.setOutputValueClass(IntWritable.class);
-
-            FileInputFormat.setInputPaths(job, new Path(igfsScheme() + inFile.toString()));
-            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
-
-            job.setJarByClass(GridHadoopWordCount2.class);
-
-            GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 1);
-
-            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
-
-            fut.get();
-
-            checkJobStatistics(jobId);
-
-            assertEquals("Use new mapper: " + useNewMapper + ", new combiner: " + useNewCombiner + ", new reducer: " +
-                useNewReducer,
-                "blue\t200000\n" +
-                "green\t150000\n" +
-                "red\t100000\n" +
-                "yellow\t70000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000")
-            );
-        }
-    }
-
-    /**
-     * Simple test job statistics.
-     *
-     * @param jobId Job id.
-     * @throws IgniteCheckedException
-     */
-    private void checkJobStatistics(GridHadoopJobId jobId) throws IgniteCheckedException, IOException {
-        GridHadoopCounters cntrs = grid(0).hadoop().counters(jobId);
-
-        GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(cntrs, null);
-
-        Map<String, SortedMap<Integer,Long>> tasks = new TreeMap<>();
-
-        Map<String, Integer> phaseOrders = new HashMap<>();
-        phaseOrders.put("submit", 0);
-        phaseOrders.put("prepare", 1);
-        phaseOrders.put("start", 2);
-        phaseOrders.put("Cstart", 3);
-        phaseOrders.put("finish", 4);
-
-        String prevTaskId = null;
-
-        long apiEvtCnt = 0;
-
-        for (T2<String, Long> evt : perfCntr.evts()) {
-            //We expect string pattern: COMBINE 1 run 7fa86a14-5a08-40e3-a7cb-98109b52a706
-            String[] parsedEvt = evt.get1().split(" ");
-
-            String taskId;
-            String taskPhase;
-
-            if ("JOB".equals(parsedEvt[0])) {
-                taskId = parsedEvt[0];
-                taskPhase = parsedEvt[1];
-            }
-            else {
-                taskId = ("COMBINE".equals(parsedEvt[0]) ? "MAP" : parsedEvt[0].substring(0, 3)) + parsedEvt[1];
-                taskPhase = ("COMBINE".equals(parsedEvt[0]) ? "C" : "") + parsedEvt[2];
-            }
-
-            if (!taskId.equals(prevTaskId))
-                tasks.put(taskId, new TreeMap<Integer,Long>());
-
-            Integer pos = phaseOrders.get(taskPhase);
-
-            assertNotNull("Invalid phase " + taskPhase, pos);
-
-            tasks.get(taskId).put(pos, evt.get2());
-
-            prevTaskId = taskId;
-
-            apiEvtCnt++;
-        }
-
-        for (Map.Entry<String ,SortedMap<Integer,Long>> task : tasks.entrySet()) {
-            Map<Integer, Long> order = task.getValue();
-
-            long prev = 0;
-
-            for (Map.Entry<Integer, Long> phase : order.entrySet()) {
-                assertTrue("Phase order of " + task.getKey() + " is invalid", phase.getValue() >= prev);
-
-                prev = phase.getValue();
-            }
-        }
-
-        final IgfsPath statPath = new IgfsPath("/xxx/yyy/zzz/" + jobId + "/performance");
-
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return igfs.exists(statPath);
-            }
-        }, 10000);
-
-        BufferedReader reader = new BufferedReader(new InputStreamReader(igfs.open(statPath)));
-
-        assertEquals(apiEvtCnt, GridHadoopTestUtils.simpleCheckJobStatFile(reader));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopPopularWordsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopPopularWordsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopPopularWordsTest.java
deleted file mode 100644
index 3e8a95a..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopPopularWordsTest.java
+++ /dev/null
@@ -1,294 +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 com.google.common.collect.*;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.Map.*;
-
-import static com.google.common.collect.Maps.*;
-import static com.google.common.collect.MinMaxPriorityQueue.*;
-import static java.util.Collections.*;
-
-/**
- * Hadoop-based 10 popular words example: all files in a given directory are tokenized and for each word longer than
- * 3 characters the number of occurrences ins calculated. Finally, 10 words with the highest occurrence count are
- * output.
- *
- * NOTE: in order to run this example on Windows please ensure that cygwin is installed and available in the system
- * path.
- */
-public class GridHadoopPopularWordsTest {
-    /** Ignite home. */
-    private static final String IGNITE_HOME = U.getIgniteHome();
-
-    /** The path to the input directory. ALl files in that directory will be processed. */
-    private static final Path BOOKS_LOCAL_DIR =
-        new Path("file:" + IGNITE_HOME, "modules/tests/java/org/apache/ignite/grid/hadoop/books");
-
-    /** The path to the output directory. THe result file will be written to this location. */
-    private static final Path RESULT_LOCAL_DIR =
-        new Path("file:" + IGNITE_HOME, "modules/tests/java/org/apache/ignite/grid/hadoop/output");
-
-    /** Popular books source dir in DFS. */
-    private static final Path BOOKS_DFS_DIR = new Path("tmp/word-count-example/in");
-
-    /** Popular books source dir in DFS. */
-    private static final Path RESULT_DFS_DIR = new Path("tmp/word-count-example/out");
-
-    /** Path to the distributed file system configuration. */
-    private static final String DFS_CFG = "examples/config/filesystem/core-site.xml";
-
-    /** Top N words to select **/
-    private static final int POPULAR_WORDS_CNT = 10;
-
-    /**
-     * For each token in the input string the mapper emits a {word, 1} pair.
-     */
-    private static class TokenizingMapper extends Mapper<LongWritable, Text, Text, IntWritable> {
-        /** Constant value. */
-        private static final IntWritable ONE = new IntWritable(1);
-
-        /** The word converted into the Text. */
-        private Text word = new Text();
-
-        /**
-         * Emits a entry where the key is the word and the value is always 1.
-         *
-         * @param key the current position in the input file (not used here)
-         * @param val the text string
-         * @param ctx mapper context
-         * @throws IOException
-         * @throws InterruptedException
-         */
-        @Override protected void map(LongWritable key, Text val, Context ctx)
-            throws IOException, InterruptedException {
-            // Get the mapped object.
-            final String line = val.toString();
-
-            // Splits the given string to words.
-            final String[] words = line.split("[^a-zA-Z0-9]");
-
-            for (final String w : words) {
-                // Only emit counts for longer words.
-                if (w.length() <= 3)
-                    continue;
-
-                word.set(w);
-
-                // Write the word into the context with the initial count equals 1.
-                ctx.write(word, ONE);
-            }
-        }
-    }
-
-    /**
-     * The reducer uses a priority queue to rank the words based on its number of occurrences.
-     */
-    private static class TopNWordsReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
-        private MinMaxPriorityQueue<Entry<Integer, String>> q;
-
-        TopNWordsReducer() {
-            q = orderedBy(reverseOrder(new Comparator<Entry<Integer, String>>() {
-                @Override public int compare(Entry<Integer, String> o1, Entry<Integer, String> o2) {
-                    return o1.getKey().compareTo(o2.getKey());
-                }
-            })).expectedSize(POPULAR_WORDS_CNT).maximumSize(POPULAR_WORDS_CNT).create();
-        }
-
-        /**
-         * This method doesn't emit anything, but just keeps track of the top N words.
-         *
-         * @param key The word.
-         * @param vals The words counts.
-         * @param ctx Reducer context.
-         * @throws IOException If failed.
-         * @throws InterruptedException If failed.
-         */
-        @Override public void reduce(Text key, Iterable<IntWritable> vals, Context ctx) throws IOException,
-            InterruptedException {
-            int sum = 0;
-
-            for (IntWritable val : vals)
-                sum += val.get();
-
-            q.add(immutableEntry(sum, key.toString()));
-        }
-
-        /**
-         * This method is called after all the word entries have been processed. It writes the accumulated
-         * statistics to the job output file.
-         *
-         * @param ctx The job context.
-         * @throws IOException If failed.
-         * @throws InterruptedException If failed.
-         */
-        @Override protected void cleanup(Context ctx) throws IOException, InterruptedException {
-            IntWritable i = new IntWritable();
-
-            Text txt = new Text();
-
-            // iterate in desc order
-            while (!q.isEmpty()) {
-                Entry<Integer, String> e = q.removeFirst();
-
-                i.set(e.getKey());
-
-                txt.set(e.getValue());
-
-                ctx.write(txt, i);
-            }
-        }
-    }
-
-    /**
-     * Configures the Hadoop MapReduce job.
-     *
-     * @return Instance of the Hadoop MapRed job.
-     * @throws IOException If failed.
-     */
-    private Job createConfigBasedHadoopJob() throws IOException {
-        Job jobCfg = new Job();
-
-        Configuration cfg = jobCfg.getConfiguration();
-
-        // Use explicit configuration of distributed file system, if provided.
-        if (DFS_CFG != null)
-            cfg.addResource(U.resolveIgniteUrl(DFS_CFG));
-
-        jobCfg.setJobName("HadoopPopularWordExample");
-        jobCfg.setJarByClass(GridHadoopPopularWordsTest.class);
-        jobCfg.setInputFormatClass(TextInputFormat.class);
-        jobCfg.setOutputKeyClass(Text.class);
-        jobCfg.setOutputValueClass(IntWritable.class);
-        jobCfg.setMapperClass(TokenizingMapper.class);
-        jobCfg.setReducerClass(TopNWordsReducer.class);
-
-        FileInputFormat.setInputPaths(jobCfg, BOOKS_DFS_DIR);
-        FileOutputFormat.setOutputPath(jobCfg, RESULT_DFS_DIR);
-
-        // Local job tracker allows the only task per wave, but text input format
-        // replaces it with the calculated value based on input split size option.
-        if ("local".equals(cfg.get("mapred.job.tracker", "local"))) {
-            // Split job into tasks using 32MB split size.
-            FileInputFormat.setMinInputSplitSize(jobCfg, 32 * 1024 * 1024);
-            FileInputFormat.setMaxInputSplitSize(jobCfg, Long.MAX_VALUE);
-        }
-
-        return jobCfg;
-    }
-
-    /**
-     * Runs the Hadoop job.
-     *
-     * @return {@code True} if succeeded, {@code false} otherwise.
-     * @throws Exception If failed.
-     */
-    private boolean runWordCountConfigBasedHadoopJob() throws Exception {
-        Job job = createConfigBasedHadoopJob();
-
-        // Distributed file system this job will work with.
-        FileSystem fs = FileSystem.get(job.getConfiguration());
-
-        X.println(">>> Using distributed file system: " + fs.getHomeDirectory());
-
-        // Prepare input and output job directories.
-        prepareDirectories(fs);
-
-        long time = System.currentTimeMillis();
-
-        // Run job.
-        boolean res = job.waitForCompletion(true);
-
-        X.println(">>> Job execution time: " + (System.currentTimeMillis() - time) / 1000 + " sec.");
-
-        // Move job results into local file system, so you can view calculated results.
-        publishResults(fs);
-
-        return res;
-    }
-
-    /**
-     * Prepare job's data: cleanup result directories that might have left over
-     * after previous runs, copy input files from the local file system into DFS.
-     *
-     * @param fs Distributed file system to use in job.
-     * @throws IOException If failed.
-     */
-    private void prepareDirectories(FileSystem fs) throws IOException {
-        X.println(">>> Cleaning up DFS result directory: " + RESULT_DFS_DIR);
-
-        fs.delete(RESULT_DFS_DIR, true);
-
-        X.println(">>> Cleaning up DFS input directory: " + BOOKS_DFS_DIR);
-
-        fs.delete(BOOKS_DFS_DIR, true);
-
-        X.println(">>> Copy local files into DFS input directory: " + BOOKS_DFS_DIR);
-
-        fs.copyFromLocalFile(BOOKS_LOCAL_DIR, BOOKS_DFS_DIR);
-    }
-
-    /**
-     * Publish job execution results into local file system, so you can view them.
-     *
-     * @param fs Distributed file sytem used in job.
-     * @throws IOException If failed.
-     */
-    private void publishResults(FileSystem fs) throws IOException {
-        X.println(">>> Cleaning up DFS input directory: " + BOOKS_DFS_DIR);
-
-        fs.delete(BOOKS_DFS_DIR, true);
-
-        X.println(">>> Cleaning up LOCAL result directory: " + RESULT_LOCAL_DIR);
-
-        fs.delete(RESULT_LOCAL_DIR, true);
-
-        X.println(">>> Moving job results into LOCAL result directory: " + RESULT_LOCAL_DIR);
-
-        fs.copyToLocalFile(true, RESULT_DFS_DIR, RESULT_LOCAL_DIR);
-    }
-
-    /**
-     * Executes a modified version of the Hadoop word count example. Here, in addition to counting the number of
-     * occurrences of the word in the source files, the N most popular words are selected.
-     *
-     * @param args None.
-     */
-    public static void main(String[] args) {
-        try {
-            new GridHadoopPopularWordsTest().runWordCountConfigBasedHadoopJob();
-        }
-        catch (Exception e) {
-            X.println(">>> Failed to run word count example: " + e.getMessage());
-        }
-
-        System.exit(0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSharedMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSharedMap.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSharedMap.java
deleted file mode 100644
index 689fb58..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSharedMap.java
+++ /dev/null
@@ -1,67 +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.jdk8.backport.*;
-
-import java.util.concurrent.*;
-
-/**
- * For tests.
- */
-public class GridHadoopSharedMap {
-    /** */
-    private static final ConcurrentMap<String, GridHadoopSharedMap> maps = new ConcurrentHashMap8<>();
-
-    /** */
-    private final ConcurrentMap<String, Object> map = new ConcurrentHashMap8<>();
-
-    /**
-     * Private.
-     */
-    private GridHadoopSharedMap() {
-        // No-op.
-    }
-
-    /**
-     * Puts object by key.
-     *
-     * @param key Key.
-     * @param val Value.
-     */
-    public <T> T put(String key, T val) {
-        Object old = map.putIfAbsent(key, val);
-
-        return old == null ? val : (T)old;
-    }
-
-    /**
-     * @param cls Class.
-     * @return Map of static fields.
-     */
-    public static GridHadoopSharedMap map(Class<?> cls) {
-        GridHadoopSharedMap m = maps.get(cls.getName());
-
-        if (m != null)
-            return m;
-
-        GridHadoopSharedMap old = maps.putIfAbsent(cls.getName(), m = new GridHadoopSharedMap());
-
-        return old == null ? m : old;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingExternalTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingExternalTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingExternalTest.java
deleted file mode 100644
index 23884ef..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingExternalTest.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.internal.processors.hadoop;
-
-/**
- * External test for sorting.
- */
-public class GridHadoopSortingExternalTest extends GridHadoopSortingTest {
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(true);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 3a2c397..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java
+++ /dev/null
@@ -1,281 +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.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Tests correct sorting.
- */
-public class GridHadoopSortingTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static final String PATH_INPUT = "/test-in";
-
-    /** */
-    private static final String PATH_OUTPUT = "/test-out";
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-
-    /**
-     * @return {@code True} if IGFS is enabled on Hadoop nodes.
-     */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSortSimple() throws Exception {
-        // Generate test data.
-        Job job = Job.getInstance();
-
-        job.setInputFormatClass(InFormat.class);
-
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        job.setMapperClass(Mapper.class);
-        job.setNumReduceTasks(0);
-
-        setupFileSystems(job.getConfiguration());
-
-        FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_INPUT));
-
-        X.printerrln("Data generation started.");
-
-        grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
-            createJobInfo(job.getConfiguration())).get(180000);
-
-        X.printerrln("Data generation complete.");
-
-        // Run main map-reduce job.
-        job = Job.getInstance();
-
-        setupFileSystems(job.getConfiguration());
-
-        job.getConfiguration().set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, JavaSerialization.class.getName() +
-            "," + WritableSerialization.class.getName());
-
-        FileInputFormat.setInputPaths(job, new Path(igfsScheme() + PATH_INPUT));
-        FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
-
-        job.setSortComparatorClass(JavaSerializationComparator.class);
-
-        job.setMapperClass(MyMapper.class);
-        job.setReducerClass(MyReducer.class);
-
-        job.setNumReduceTasks(2);
-
-        job.setMapOutputKeyClass(UUID.class);
-        job.setMapOutputValueClass(NullWritable.class);
-
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        X.printerrln("Job started.");
-
-        grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 2),
-            createJobInfo(job.getConfiguration())).get(180000);
-
-        X.printerrln("Job complete.");
-
-        // Check result.
-        Path outDir = new Path(igfsScheme() + PATH_OUTPUT);
-
-        AbstractFileSystem fs = AbstractFileSystem.get(new URI(igfsScheme()), job.getConfiguration());
-
-        for (FileStatus file : fs.listStatus(outDir)) {
-            X.printerrln("__ file: " + file);
-
-            if (file.getLen() == 0)
-                continue;
-
-            FSDataInputStream in = fs.open(file.getPath());
-
-            Scanner sc = new Scanner(in);
-
-            UUID prev = null;
-
-            while(sc.hasNextLine()) {
-                UUID next = UUID.fromString(sc.nextLine());
-
-//                X.printerrln("___ check: " + next);
-
-                if (prev != null)
-                    assertTrue(prev.compareTo(next) < 0);
-
-                prev = next;
-            }
-        }
-    }
-
-    public static class InFormat extends InputFormat<Text, NullWritable> {
-        /** {@inheritDoc} */
-        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException, InterruptedException {
-            List<InputSplit> res = new ArrayList<>();
-
-            FakeSplit split = new FakeSplit(20);
-
-            for (int i = 0; i < 10; i++)
-                res.add(split);
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public RecordReader<Text, NullWritable> createRecordReader(final InputSplit split,
-            TaskAttemptContext ctx) throws IOException, InterruptedException {
-            return new RecordReader<Text, NullWritable>() {
-                /** */
-                int cnt;
-
-                /** */
-                Text txt = new Text();
-
-                @Override public void initialize(InputSplit split, TaskAttemptContext ctx) {
-                    // No-op.
-                }
-
-                @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-                    return ++cnt <= split.getLength();
-                }
-
-                @Override public Text getCurrentKey() {
-                    txt.set(UUID.randomUUID().toString());
-
-//                    X.printerrln("___ read: " + txt);
-
-                    return txt;
-                }
-
-                @Override public NullWritable getCurrentValue() {
-                    return NullWritable.get();
-                }
-
-                @Override public float getProgress() throws IOException, InterruptedException {
-                    return (float)cnt / split.getLength();
-                }
-
-                @Override public void close() {
-                    // No-op.
-                }
-            };
-        }
-    }
-
-    public static class MyMapper extends Mapper<LongWritable, Text, UUID, NullWritable> {
-        /** {@inheritDoc} */
-        @Override protected void map(LongWritable key, Text val, Context ctx) throws IOException, InterruptedException {
-//            X.printerrln("___ map: " + val);
-
-            ctx.write(UUID.fromString(val.toString()), NullWritable.get());
-        }
-    }
-
-    public static class MyReducer extends Reducer<UUID, NullWritable, Text, NullWritable> {
-        /** */
-        private Text text = new Text();
-
-        /** {@inheritDoc} */
-        @Override protected void reduce(UUID key, Iterable<NullWritable> vals, Context ctx)
-            throws IOException, InterruptedException {
-//            X.printerrln("___ rdc: " + key);
-
-            text.set(key.toString());
-
-            ctx.write(text, NullWritable.get());
-        }
-    }
-
-    public static class FakeSplit extends InputSplit implements Writable {
-        /** */
-        private static final String[] HOSTS = {"127.0.0.1"};
-
-        /** */
-        private int len;
-
-        /**
-         * @param len Length.
-         */
-        public FakeSplit(int len) {
-            this.len = len;
-        }
-
-        /**
-         *
-         */
-        public FakeSplit() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getLength() throws IOException, InterruptedException {
-            return len;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String[] getLocations() throws IOException, InterruptedException {
-            return HOSTS;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(DataOutput out) throws IOException {
-            out.writeInt(len);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readFields(DataInput in) throws IOException {
-            len = in.readInt();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 0b15a2c..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 = GridHadoopUtils.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", GridHadoopUtils.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/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopStartup.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopStartup.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopStartup.java
deleted file mode 100644
index 6cc7635..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopStartup.java
+++ /dev/null
@@ -1,55 +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.ignite.*;
-import org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem;
-import org.apache.ignite.internal.util.typedef.*;
-
-/**
- * Hadoop node startup.
- */
-public class GridHadoopStartup {
-    /**
-     * @param args Arguments.
-     */
-    public static void main(String[] args) {
-        G.start("config/hadoop/default-config.xml");
-    }
-
-    /**
-     * @return Configuration for job run.
-     */
-    @SuppressWarnings("UnnecessaryFullyQualifiedName")
-    public static Configuration configuration() {
-        Configuration cfg = new Configuration();
-
-        cfg.set("fs.defaultFS", "igfs://igfs@localhost");
-
-        cfg.set("fs.igfs.impl", org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem.class.getName());
-        cfg.set("fs.AbstractFileSystem.igfs.impl", IgfsHadoopFileSystem.class.getName());
-
-        cfg.set("dfs.client.block.write.replace-datanode-on-failure.policy", "NEVER");
-
-        cfg.set("mapreduce.framework.name", "ignite");
-        cfg.set("mapreduce.jobtracker.address", "localhost:11211");
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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
deleted file mode 100644
index 40546bb..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
+++ /dev/null
@@ -1,551 +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.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.igfs.hadoop.v1.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.testframework.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
-
-/**
- * Tests map-reduce task execution basics.
- */
-public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static GridHadoopSharedMap m = GridHadoopSharedMap.map(GridHadoopTaskExecutionSelfTest.class);
-
-    /** Line count. */
-    private static final AtomicInteger totalLineCnt = m.put("totalLineCnt", new AtomicInteger());
-
-    /** Executed tasks. */
-    private static final AtomicInteger executedTasks = m.put("executedTasks", new AtomicInteger());
-
-    /** Cancelled tasks. */
-    private static final AtomicInteger cancelledTasks = m.put("cancelledTasks", new AtomicInteger());
-
-    /** Working directory of each task. */
-    private static final Map<String, String> taskWorkDirs = m.put("taskWorkDirs",
-        new ConcurrentHashMap<String, String>());
-
-    /** Mapper id to fail. */
-    private static final AtomicInteger failMapperId = m.put("failMapperId", new AtomicInteger());
-
-    /** Number of splits of the current input. */
-    private static final AtomicInteger splitsCount = m.put("splitsCount", new AtomicInteger());
-
-    /** Test param. */
-    private static final String MAP_WRITE = "test.map.write";
-
-
-    /** {@inheritDoc} */
-    @Override public IgfsConfiguration igfsConfiguration() {
-        IgfsConfiguration cfg = super.igfsConfiguration();
-
-        cfg.setFragmentizerEnabled(false);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean igfsEnabled() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrids(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-
-        super.afterTestsStopped();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        grid(0).fileSystem(igfsName).format();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration hadoopConfiguration(String gridName) {
-        GridHadoopConfiguration cfg = super.hadoopConfiguration(gridName);
-
-        cfg.setMaxParallelTasks(5);
-        cfg.setExternalExecution(false);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapRun() throws Exception {
-        int lineCnt = 10000;
-        String fileName = "/testFile";
-
-        prepareFile(fileName, lineCnt);
-
-        totalLineCnt.set(0);
-        taskWorkDirs.clear();
-
-        Configuration cfg = new Configuration();
-
-        cfg.setStrings("fs.igfs.impl", IgfsHadoopFileSystem.class.getName());
-
-        Job job = Job.getInstance(cfg);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(TestMapper.class);
-
-        job.setNumReduceTasks(0);
-
-        job.setInputFormatClass(TextInputFormat.class);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
-
-        job.setJarByClass(getClass());
-
-        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
-                createJobInfo(job.getConfiguration()));
-
-        fut.get();
-
-        assertEquals(lineCnt, totalLineCnt.get());
-
-        assertEquals(32, taskWorkDirs.size());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapCombineRun() throws Exception {
-        int lineCnt = 10001;
-        String fileName = "/testFile";
-
-        prepareFile(fileName, lineCnt);
-
-        totalLineCnt.set(0);
-        taskWorkDirs.clear();
-
-        Configuration cfg = new Configuration();
-
-        cfg.setStrings("fs.igfs.impl", IgfsHadoopFileSystem.class.getName());
-        cfg.setBoolean(MAP_WRITE, true);
-
-        Job job = Job.getInstance(cfg);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(TestMapper.class);
-        job.setCombinerClass(TestCombiner.class);
-        job.setReducerClass(TestReducer.class);
-
-        job.setNumReduceTasks(2);
-
-        job.setInputFormatClass(TextInputFormat.class);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output"));
-
-        job.setJarByClass(getClass());
-
-        GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 2);
-
-        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
-
-        fut.get();
-
-        assertEquals(lineCnt, totalLineCnt.get());
-
-        assertEquals(34, taskWorkDirs.size());
-
-        for (int g = 0; g < gridCount(); g++)
-            grid(g).hadoop().finishFuture(jobId).get();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapperException() throws Exception {
-        prepareFile("/testFile", 1000);
-
-        Configuration cfg = new Configuration();
-
-        cfg.setStrings("fs.igfs.impl", IgfsHadoopFileSystem.class.getName());
-
-        Job job = Job.getInstance(cfg);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(FailMapper.class);
-
-        job.setNumReduceTasks(0);
-
-        job.setInputFormatClass(TextInputFormat.class);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
-
-        job.setJarByClass(getClass());
-
-        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 3),
-                createJobInfo(job.getConfiguration()));
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fut.get();
-
-                return null;
-            }
-        }, IgniteCheckedException.class, null);
-    }
-
-    /**
-     * @param fileName File name.
-     * @param lineCnt Line count.
-     * @throws Exception If failed.
-     */
-    private void prepareFile(String fileName, int lineCnt) throws Exception {
-        IgniteFs igfs = grid(0).fileSystem(igfsName);
-
-        try (OutputStream os = igfs.create(new IgfsPath(fileName), true)) {
-            PrintWriter w = new PrintWriter(new OutputStreamWriter(os));
-
-            for (int i = 0; i < lineCnt; i++)
-                w.print("Hello, Hadoop map-reduce!\n");
-
-            w.flush();
-        }
-    }
-
-    /**
-     * Prepare job with mappers to cancel.
-     * @return Fully configured job.
-     * @throws Exception If fails.
-     */
-    private Configuration prepareJobForCancelling() throws Exception {
-        prepareFile("/testFile", 1500);
-
-        executedTasks.set(0);
-        cancelledTasks.set(0);
-        failMapperId.set(0);
-        splitsCount.set(0);
-
-        Configuration cfg = new Configuration();
-
-        setupFileSystems(cfg);
-
-        Job job = Job.getInstance(cfg);
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
-
-        job.setMapperClass(CancellingTestMapper.class);
-
-        job.setNumReduceTasks(0);
-
-        job.setInputFormatClass(InFormat.class);
-
-        FileInputFormat.setInputPaths(job, new Path("igfs://:" + getTestGridName(0) + "@/"));
-        FileOutputFormat.setOutputPath(job, new Path("igfs://:" + getTestGridName(0) + "@/output/"));
-
-        job.setJarByClass(getClass());
-
-        return job.getConfiguration();
-    }
-
-    /**
-     * Test input format.
-     */
-    private static class InFormat extends TextInputFormat {
-        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException {
-            List<InputSplit> res = super.getSplits(ctx);
-
-            splitsCount.set(res.size());
-
-            X.println("___ split of input: " + splitsCount.get());
-
-            return res;
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTaskCancelling() throws Exception {
-        Configuration cfg = prepareJobForCancelling();
-
-        GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 1);
-
-        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(cfg));
-
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return splitsCount.get() > 0;
-            }
-        }, 20000)) {
-            U.dumpThreads(log);
-
-            assertTrue(false);
-        }
-
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return executedTasks.get() == splitsCount.get();
-            }
-        }, 20000)) {
-            U.dumpThreads(log);
-
-            assertTrue(false);
-        }
-
-        // Fail mapper with id "1", cancels others
-        failMapperId.set(1);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fut.get();
-
-                return null;
-            }
-        }, IgniteCheckedException.class, null);
-
-        assertEquals(executedTasks.get(), cancelledTasks.get() + 1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testJobKill() throws Exception {
-        Configuration cfg = prepareJobForCancelling();
-
-        GridHadoop hadoop = grid(0).hadoop();
-
-        GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 1);
-
-        //Kill unknown job.
-        boolean killRes = hadoop.kill(jobId);
-
-        assertFalse(killRes);
-
-        final IgniteInternalFuture<?> fut = hadoop.submit(jobId, createJobInfo(cfg));
-
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return splitsCount.get() > 0;
-            }
-        }, 20000)) {
-            U.dumpThreads(log);
-
-            assertTrue(false);
-        }
-
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                X.println("___ executed tasks: " + executedTasks.get());
-
-                return executedTasks.get() == splitsCount.get();
-            }
-        }, 20000)) {
-            U.dumpThreads(log);
-
-            fail();
-        }
-
-        //Kill really ran job.
-        killRes = hadoop.kill(jobId);
-
-        assertTrue(killRes);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fut.get();
-
-                return null;
-            }
-        }, IgniteCheckedException.class, null);
-
-        assertEquals(executedTasks.get(), cancelledTasks.get());
-
-        //Kill the same job again.
-        killRes = hadoop.kill(jobId);
-
-        assertTrue(killRes);
-    }
-
-    private static class CancellingTestMapper extends Mapper<Object, Text, Text, IntWritable> {
-        private int mapperId;
-
-        /** {@inheritDoc} */
-        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
-            mapperId = executedTasks.incrementAndGet();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void run(Context ctx) throws IOException, InterruptedException {
-            try {
-                super.run(ctx);
-            }
-            catch (GridHadoopTaskCancelledException e) {
-                cancelledTasks.incrementAndGet();
-
-                throw e;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            if (mapperId == failMapperId.get())
-                throw new IOException();
-
-            Thread.sleep(1000);
-        }
-    }
-
-    /**
-     * Test failing mapper.
-     */
-    private static class FailMapper extends Mapper<Object, Text, Text, IntWritable> {
-        /** {@inheritDoc} */
-        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            throw new IOException("Expected");
-        }
-    }
-
-    /**
-     * Mapper calculates number of lines.
-     */
-    private static class TestMapper extends Mapper<Object, Text, Text, IntWritable> {
-        /** Writable integer constant of '1'. */
-        private static final IntWritable ONE = new IntWritable(1);
-
-        /** Line count constant. */
-        public static final Text LINE_COUNT = new Text("lineCount");
-
-        /** {@inheritDoc} */
-        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
-            X.println("___ Mapper: " + ctx.getTaskAttemptID());
-
-            String taskId = ctx.getTaskAttemptID().toString();
-
-            LocalFileSystem locFs = FileSystem.getLocal(ctx.getConfiguration());
-
-            String workDir = locFs.getWorkingDirectory().toString();
-
-            assertNull(taskWorkDirs.put(workDir, taskId));
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void map(Object key, Text val, Context ctx) throws IOException, InterruptedException {
-            if (ctx.getConfiguration().getBoolean(MAP_WRITE, false))
-                ctx.write(LINE_COUNT, ONE);
-            else
-                totalLineCnt.incrementAndGet();
-        }
-    }
-
-    /**
-     * Combiner calculates number of lines.
-     */
-    private static class TestCombiner extends Reducer<Text, IntWritable, Text, IntWritable> {
-        /** */
-        IntWritable sum = new IntWritable();
-
-        /** {@inheritDoc} */
-        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
-            X.println("___ Combiner: ");
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
-            InterruptedException {
-            int lineCnt = 0;
-
-            for (IntWritable value : values)
-                lineCnt += value.get();
-
-            sum.set(lineCnt);
-
-            X.println("___ combo: " + lineCnt);
-
-            ctx.write(key, sum);
-        }
-    }
-
-    /**
-     * Combiner calculates number of lines.
-     */
-    private static class TestReducer extends Reducer<Text, IntWritable, Text, IntWritable> {
-        /** */
-        IntWritable sum = new IntWritable();
-
-        /** {@inheritDoc} */
-        @Override protected void setup(Context ctx) throws IOException, InterruptedException {
-            X.println("___ Reducer: " + ctx.getTaskAttemptID());
-
-            String taskId = ctx.getTaskAttemptID().toString();
-            String workDir = FileSystem.getLocal(ctx.getConfiguration()).getWorkingDirectory().toString();
-
-            assertNull(taskWorkDirs.put(workDir, taskId));
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void reduce(Text key, Iterable<IntWritable> values, Context ctx) throws IOException,
-            InterruptedException {
-            int lineCnt = 0;
-
-            for (IntWritable value : values) {
-                lineCnt += value.get();
-
-                X.println("___ rdcr: " + value.get());
-            }
-
-            sum.set(lineCnt);
-
-            ctx.write(key, sum);
-
-            X.println("___ RDCR SUM: " + lineCnt);
-
-            totalLineCnt.addAndGet(lineCnt);
-        }
-    }
-}


[13/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index 4ad74d0..2994107 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -2,14 +2,15 @@ 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.igfs.hadoop.*;
-import org.apache.ignite.igfs.hadoop.v1.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
+import org.apache.ignite.hadoop.fs.*;
+import org.apache.ignite.hadoop.fs.v1.*;
 import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -221,7 +222,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
      * Starts secondary IGFS
      */
     private void startSecondary() {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("partitioned");
         igfsCfg.setMetaCacheName("replicated");
@@ -259,7 +260,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
 
         cfg.setCommunicationSpi(communicationSpi());
@@ -295,7 +296,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
 
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(cacheConfiguration());
-        cfg.setIgfsConfiguration(igfsConfiguration(gridName));
+        cfg.setFileSystemConfiguration(fsConfiguration(gridName));
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
         cfg.setCommunicationSpi(communicationSpi());
 
@@ -336,8 +337,8 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
      * @param gridName Grid name.
      * @return IGFS configuration.
      */
-    protected IgfsConfiguration igfsConfiguration(String gridName) throws IgniteCheckedException {
-        IgfsConfiguration cfg = new IgfsConfiguration();
+    protected FileSystemConfiguration fsConfiguration(String gridName) throws IgniteCheckedException {
+        FileSystemConfiguration cfg = new FileSystemConfiguration();
 
         cfg.setDataCacheName("partitioned");
         cfg.setMetaCacheName("replicated");
@@ -347,7 +348,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
 
         if (mode != PRIMARY)
             cfg.setSecondaryFileSystem(
-                new IgfsHadoopFileSystemWrapper(secondaryFsUriStr, secondaryConfFullPath));
+                new IgniteHadoopIgfsSecondaryFileSystem(secondaryFsUriStr, secondaryConfFullPath));
 
         cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
 
@@ -462,10 +463,10 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
 
         if (authority != null) {
             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;
@@ -477,10 +478,10 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
      * @param cfg the configuration to set parameters into.
      */
     private static void setImplClasses(Configuration cfg) {
-        cfg.set("fs.igfs.impl", IgfsHadoopFileSystem.class.getName());
+        cfg.set("fs.igfs.impl", IgniteHadoopFileSystem.class.getName());
 
         cfg.set("fs.AbstractFileSystem.igfs.impl",
-            org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem.class.getName());
+            org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem.class.getName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 29696bf..f6f5bae 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
@@ -20,7 +20,7 @@ package org.apache.ignite.igfs;
 import junit.framework.*;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.hadoop.*;
+import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -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");
 
@@ -76,8 +76,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class ShmemPrivate extends IgfsEventsAbstractSelfTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
                 put("type", "shmem");
@@ -93,8 +93,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class LoopbackPrivate extends IgfsEventsAbstractSelfTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
                 put("type", "tcp");
@@ -110,13 +110,13 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public abstract static class PrimarySecondaryTest extends IgfsEventsAbstractSelfTest {
         /** Secondary file system. */
-        private static IgniteFs igfsSec;
+        private static IgniteFileSystem igfsSec;
 
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
-            igfsCfg.setSecondaryFileSystem(new IgfsHadoopFileSystemWrapper(
+            igfsCfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
                 "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/",
                 "modules/core/src/test/config/hadoop/core-site-secondary.xml"));
 
@@ -126,8 +126,8 @@ public class IgfsEventsTestSuite extends TestSuite {
         /**
          * @return IGFS configuration for secondary file system.
          */
-        protected IgfsConfiguration getSecondaryIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        protected FileSystemConfiguration getSecondaryIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setName("igfs-secondary");
             igfsCfg.setDefaultMode(PRIMARY);
@@ -167,7 +167,7 @@ public class IgfsEventsTestSuite extends TestSuite {
          * @return Secondary file system handle.
          * @throws Exception If failed.
          */
-        @Nullable private IgniteFs startSecondary() throws Exception {
+        @Nullable private IgniteFileSystem startSecondary() throws Exception {
             IgniteConfiguration cfg = getConfiguration("grid-secondary", getSecondaryIgfsConfiguration());
 
             cfg.setLocalHost("127.0.0.1");
@@ -184,8 +184,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class ShmemDualSync extends PrimarySecondaryTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setDefaultMode(DUAL_SYNC);
 
@@ -198,8 +198,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class ShmemDualAsync extends PrimarySecondaryTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setDefaultMode(DUAL_ASYNC);
 
@@ -212,10 +212,10 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public abstract static class LoopbackPrimarySecondaryTest extends PrimarySecondaryTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
-            igfsCfg.setSecondaryFileSystem(new IgfsHadoopFileSystemWrapper(
+            igfsCfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
                 "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/",
                 "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml"));
 
@@ -223,8 +223,8 @@ public class IgfsEventsTestSuite extends TestSuite {
         }
 
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getSecondaryIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getSecondaryIgfsConfiguration();
+        @Override protected FileSystemConfiguration getSecondaryIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getSecondaryIgfsConfiguration();
 
             igfsCfg.setName("igfs-secondary");
             igfsCfg.setDefaultMode(PRIMARY);
@@ -242,8 +242,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class LoopbackDualSync extends LoopbackPrimarySecondaryTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setDefaultMode(DUAL_SYNC);
 
@@ -256,8 +256,8 @@ public class IgfsEventsTestSuite extends TestSuite {
      */
     public static class LoopbackDualAsync extends LoopbackPrimarySecondaryTest {
         /** {@inheritDoc} */
-        @Override protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-            IgfsConfiguration igfsCfg = super.getIgfsConfiguration();
+        @Override protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+            FileSystemConfiguration igfsCfg = super.getIgfsConfiguration();
 
             igfsCfg.setDefaultMode(DUAL_ASYNC);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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 9f9a6d8..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.igfs.hadoop.*;
-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 IgfsHadoopFileSystemWrapper(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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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;
-    }
-}


[10/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemHandshakeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemHandshakeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemHandshakeSelfTest.java
deleted file mode 100644
index 137db6d..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemHandshakeSelfTest.java
+++ /dev/null
@@ -1,311 +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.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.typedef.*;
-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 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.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.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
-
-/**
- * Tests for IGFS file system handshake.
- */
-public class IgfsHadoopFileSystemHandshakeSelfTest extends IgfsCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Grid name. */
-    private static final String GRID_NAME = "grid";
-
-    /** IGFS name. */
-    private static final String IGFS_NAME = "igfs";
-
-    /** IGFS path. */
-    private static final IgfsPath PATH = new IgfsPath("/path");
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids(true);
-    }
-
-    /**
-     * Tests for Grid and IGFS having normal names.
-     *
-     * @throws Exception If failed.
-     */
-    public void testHandshake() throws Exception {
-        startUp(false, false);
-
-        checkValid(IGFS_NAME + ":" + GRID_NAME + "@");
-        checkValid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
-        checkValid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(IGFS_NAME + "@");
-        checkInvalid(IGFS_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(":" + GRID_NAME + "@");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid("");
-        checkInvalid("127.0.0.1");
-        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
-    }
-
-    /**
-     * Tests for Grid having {@code null} name and IGFS having normal name.
-     *
-     * @throws Exception If failed.
-     */
-    public void testHandshakeDefaultGrid() throws Exception {
-        startUp(true, false);
-
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkValid(IGFS_NAME + "@");
-        checkValid(IGFS_NAME + "@127.0.0.1");
-        checkValid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(":" + GRID_NAME + "@");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid("");
-        checkInvalid("127.0.0.1");
-        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
-    }
-
-    /**
-     * Tests for Grid having normal name and IGFS having {@code null} name.
-     *
-     * @throws Exception If failed.
-     */
-    public void testHandshakeDefaultIgfs() throws Exception {
-        startUp(false, true);
-
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(IGFS_NAME + "@");
-        checkInvalid(IGFS_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkValid(":" + GRID_NAME + "@");
-        checkValid(":" + GRID_NAME + "@127.0.0.1");
-        checkValid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid("");
-        checkInvalid("127.0.0.1");
-        checkInvalid("127.0.0.1:" + DFLT_IPC_PORT);
-    }
-
-    /**
-     * Tests for Grid having {@code null} name and IGFS having {@code null} name.
-     *
-     * @throws Exception If failed.
-     */
-    public void testHandshakeDefaultGridDefaultIgfs() throws Exception {
-        startUp(true, true);
-
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(IGFS_NAME + "@");
-        checkInvalid(IGFS_NAME + "@127.0.0.1");
-        checkInvalid(IGFS_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkInvalid(":" + GRID_NAME + "@");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1");
-        checkInvalid(":" + GRID_NAME + "@127.0.0.1:" + DFLT_IPC_PORT);
-
-        checkValid("");
-        checkValid("127.0.0.1");
-        checkValid("127.0.0.1:" + DFLT_IPC_PORT);
-    }
-
-    /**
-     * Perform startup.
-     *
-     * @param dfltGridName Default Grid name.
-     * @param dfltIgfsName Default IGFS name.
-     * @throws Exception If failed.
-     */
-    private void startUp(boolean dfltGridName, boolean dfltIgfsName) throws Exception {
-        Ignite ignite = G.start(gridConfiguration(dfltGridName, dfltIgfsName));
-
-        IgniteFs igfs = ignite.fileSystem(dfltIgfsName ? null : IGFS_NAME);
-
-        igfs.mkdirs(PATH);
-    }
-
-    /**
-     * Create Grid configuration.
-     *
-     * @param dfltGridName Default Grid name.
-     * @param dfltIgfsName Default IGFS name.
-     * @return Grid configuration.
-     * @throws Exception If failed.
-     */
-    private IgniteConfiguration gridConfiguration(boolean dfltGridName, boolean dfltIgfsName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(dfltGridName ? null : GRID_NAME);
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setConnectorConfiguration(null);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-
-        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
-
-        commSpi.setSharedMemoryPort(-1);
-
-        cfg.setCommunicationSpi(commSpi);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("replicated");
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        metaCacheCfg.setQueryIndexEnabled(false);
-        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
-
-        dataCacheCfg.setName("partitioned");
-        dataCacheCfg.setCacheMode(PARTITIONED);
-        dataCacheCfg.setDistributionMode(PARTITIONED_ONLY);
-        dataCacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        dataCacheCfg.setBackups(0);
-        dataCacheCfg.setQueryIndexEnabled(false);
-        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        cfg.setCacheConfiguration(metaCacheCfg, dataCacheCfg);
-
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("replicated");
-        igfsCfg.setName(dfltIgfsName ? null : IGFS_NAME);
-        igfsCfg.setPrefetchBlocks(1);
-        igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
-            put("type", "tcp");
-            put("port", String.valueOf(DFLT_IPC_PORT));
-        }});
-
-        igfsCfg.setManagementPort(-1);
-        igfsCfg.setBlockSize(512 * 1024);
-
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        return cfg;
-    }
-
-    /**
-     * Check valid file system endpoint.
-     *
-     * @param authority Authority.
-     * @throws Exception If failed.
-     */
-    private void checkValid(String authority) throws Exception {
-        FileSystem fs = fileSystem(authority);
-
-        assert fs.exists(new Path(PATH.toString()));
-    }
-
-    /**
-     * Check invalid file system endpoint.
-     *
-     * @param authority Authority.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    private void checkInvalid(final String authority) throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fileSystem(authority);
-
-                return null;
-            }
-        }, IOException.class, null);
-    }
-
-    /**
-     *
-     *
-     * @param authority Authority.
-     * @return File system.
-     * @throws Exception If failed.
-     */
-    private static FileSystem fileSystem(String authority) throws Exception {
-        return FileSystem.get(new URI("igfs://" + authority + "/"), configuration(authority));
-    }
-
-    /**
-     * Create configuration for test.
-     *
-     * @param authority Authority.
-     * @return Configuration.
-     */
-    private static Configuration configuration(String authority) {
-        Configuration cfg = new Configuration();
-
-        cfg.set("fs.defaultFS", "igfs://" + authority + "/");
-        cfg.set("fs.igfs.impl", org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem.class.getName());
-        cfg.set("fs.AbstractFileSystem.igfs.impl",
-            IgfsHadoopFileSystem.class.getName());
-
-        cfg.setBoolean("fs.igfs.impl.disable.cache", true);
-
-        cfg.setBoolean(String.format(PARAM_IGFS_ENDPOINT_NO_EMBED, authority), true);
-        cfg.setBoolean(String.format(PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority), true);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemIpcCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemIpcCacheSelfTest.java
deleted file mode 100644
index a6357f8..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemIpcCacheSelfTest.java
+++ /dev/null
@@ -1,207 +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.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.ipc.shmem.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-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 java.lang.reflect.*;
-import java.net.*;
-import java.util.*;
-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.*;
-
-/**
- * IPC cache test.
- */
-public class IgfsHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Path to test hadoop configuration. */
-    private static final String HADOOP_FS_CFG = "modules/core/src/test/config/hadoop/core-site.xml";
-
-    /** Group size. */
-    public static final int GRP_SIZE = 128;
-
-    /** Started grid counter. */
-    private static int cnt;
-
-    /** {@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);
-
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("replicated");
-        igfsCfg.setName("igfs");
-        igfsCfg.setManagementPort(IgfsConfiguration.DFLT_MGMT_PORT + cnt);
-
-        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
-            put("type", "shmem");
-            put("port", String.valueOf(IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT + cnt));
-        }});
-
-        igfsCfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
-
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setCacheConfiguration(cacheConfiguration());
-
-        cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-
-        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
-
-        commSpi.setSharedMemoryPort(-1);
-
-        cfg.setCommunicationSpi(commSpi);
-
-        cnt++;
-
-        return cfg;
-    }
-
-    /**
-     * Gets cache configuration.
-     *
-     * @return Cache configuration.
-     */
-    private CacheConfiguration[] cacheConfiguration() {
-        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};
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(4);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        G.stopAll(true);
-    }
-
-    /**
-     * Test how IPC cache map works.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public void testIpcCache() throws Exception {
-        Field cacheField = IgfsHadoopIpcIo.class.getDeclaredField("ipcCache");
-
-        cacheField.setAccessible(true);
-
-        Field activeCntField = IgfsHadoopIpcIo.class.getDeclaredField("activeCnt");
-
-        activeCntField.setAccessible(true);
-
-        Map<String, IgfsHadoopIpcIo> cache = (Map<String, IgfsHadoopIpcIo>)cacheField.get(null);
-
-        String name = "igfs:" + getTestGridName(0) + "@";
-
-        Configuration cfg = new Configuration();
-
-        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);
-
-        // Ensure that existing IO is reused.
-        FileSystem fs1 = FileSystem.get(new URI("igfs://" + name + "/"), cfg);
-
-        assertEquals(1, cache.size());
-
-        IgfsHadoopIpcIo io = null;
-
-        System.out.println("CACHE: " + cache);
-
-        for (String key : cache.keySet()) {
-            if (key.contains("10500")) {
-                io = cache.get(key);
-
-                break;
-            }
-        }
-
-        assert io != null;
-
-        assertEquals(1, ((AtomicInteger)activeCntField.get(io)).get());
-
-        // Ensure that when IO is used by multiple file systems and one of them is closed, IO is not stopped.
-        FileSystem fs2 = FileSystem.get(new URI("igfs://" + name + "/abc"), cfg);
-
-        assertEquals(1, cache.size());
-        assertEquals(2, ((AtomicInteger)activeCntField.get(io)).get());
-
-        fs2.close();
-
-        assertEquals(1, cache.size());
-        assertEquals(1, ((AtomicInteger)activeCntField.get(io)).get());
-
-        Field stopField = IgfsHadoopIpcIo.class.getDeclaredField("stopping");
-
-        stopField.setAccessible(true);
-
-        assert !(Boolean)stopField.get(io);
-
-        // Ensure that IO is stopped when nobody else is need it.
-        fs1.close();
-
-        assert cache.isEmpty();
-
-        assert (Boolean)stopField.get(io);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerSelfTest.java
deleted file mode 100644
index 2e3a3ec..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerSelfTest.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.igfs.IgfsMode.*;
-import static org.apache.ignite.internal.igfs.common.IgfsLogger.*;
-
-/**
- * Grid IGFS client logger test.
- */
-public class IgfsHadoopFileSystemLoggerSelfTest extends IgfsCommonAbstractTest {
-    /** Path string. */
-    private static final String PATH_STR = "/dir1/dir2/file;test";
-
-    /** Path string with escaped semicolons. */
-    private static final String PATH_STR_ESCAPED = PATH_STR.replace(';', '~');
-
-    /** Path. */
-    private static final IgfsPath PATH = new IgfsPath(PATH_STR);
-
-    /** IGFS name. */
-    private static final String IGFS_NAME = "igfs";
-
-    /** Log file path. */
-    private static final String LOG_DIR = U.getIgniteHome();
-
-    /** Endpoint address. */
-    private static final String ENDPOINT = "localhost:10500";
-
-    /** Log file name. */
-    private static final String LOG_FILE = LOG_DIR + File.separator + "igfs-log-" + IGFS_NAME + "-" + U.jvmPid() +
-        ".csv";
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        removeLogs();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        removeLogs();
-    }
-
-    /**
-     * Remove existing logs.
-     *
-     * @throws Exception If failed.
-     */
-    private void removeLogs() throws Exception {
-        File dir = new File(LOG_DIR);
-
-        File[] logs = dir.listFiles(new FilenameFilter() {
-            @Override public boolean accept(File dir, String name) {
-                return name.startsWith("igfs-log-");
-            }
-        });
-
-        for (File log : logs)
-            log.delete();
-    }
-
-    /**
-     * Ensure correct static loggers creation/removal as well as file creation.
-     *
-     * @throws Exception If failed.
-     */
-    public void testCreateDelete() throws Exception {
-        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        IgfsLogger sameLog0 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        // Loggers for the same endpoint must be the same object.
-        assert log == sameLog0;
-
-        IgfsLogger otherLog = IgfsLogger.logger("other" + ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        // Logger for another endpoint must be different.
-        assert log != otherLog;
-
-        otherLog.close();
-
-        log.logDelete(PATH, PRIMARY, false);
-
-        log.close();
-
-        File logFile = new File(LOG_FILE);
-
-        // When there are multiple loggers, closing one must not force flushing.
-        assert !logFile.exists();
-
-        IgfsLogger sameLog1 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        assert sameLog0 == sameLog1;
-
-        sameLog0.close();
-
-        assert !logFile.exists();
-
-        sameLog1.close();
-
-        // When we cloe the last logger, it must flush data to disk.
-        assert logFile.exists();
-
-        logFile.delete();
-
-        IgfsLogger sameLog2 = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        // This time we expect new logger instance to be created.
-        assert sameLog0 != sameLog2;
-
-        sameLog2.close();
-
-        // As we do not add any records to the logger, we do not expect flushing.
-        assert !logFile.exists();
-    }
-
-    /**
-     * Test read operations logging.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLogRead() throws Exception {
-        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        log.logOpen(1, PATH, PRIMARY, 2, 3L);
-        log.logRandomRead(1, 4L, 5);
-        log.logSeek(1, 6L);
-        log.logSkip(1, 7L);
-        log.logMark(1, 8L);
-        log.logReset(1);
-        log.logCloseIn(1, 9L, 10L, 11);
-
-        log.close();
-
-        checkLog(
-            new SB().a(U.jvmPid() + d() + TYPE_OPEN_IN + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 1 + d() + 2 +
-                d() + 3 + d(14)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_RANDOM_READ + d(3) + 1 + d(7) + 4 + d() + 5 + d(8)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_SEEK + d(3) + 1 + d(7) + 6 + d(9)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_SKIP + d(3) + 1 + d(9) + 7 + d(7)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_MARK + d(3) + 1 + d(10) + 8 + d(6)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_RESET + d(3) + 1 + d(16)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_CLOSE_IN + d(3) + 1 + d(11) + 9 + d() + 10 + d() + 11 + d(3)).toString()
-        );
-    }
-
-    /**
-     * Test write operations logging.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLogWrite() throws Exception {
-        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        log.logCreate(1, PATH, PRIMARY, true, 2, new Integer(3).shortValue(), 4L);
-        log.logAppend(2, PATH, PRIMARY, 8);
-        log.logCloseOut(2, 9L, 10L, 11);
-
-        log.close();
-
-        checkLog(
-            new SB().a(U.jvmPid() + d() + TYPE_OPEN_OUT + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 1 + d() +
-                2 + d(2) + 0 + d() + 1 + d() + 3 + d() + 4 + d(10)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_OPEN_OUT + d() + PATH_STR_ESCAPED + d() + PRIMARY + d() + 2 + d() +
-                8 + d(2) + 1 + d(13)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_CLOSE_OUT + d(3) + 2 + d(11) + 9 + d() + 10 + d() + 11 + d(3))
-                .toString()
-        );
-    }
-
-    /**
-     * Test miscellaneous operations logging.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testLogMisc() throws Exception {
-        IgfsLogger log = IgfsLogger.logger(ENDPOINT, IGFS_NAME, LOG_DIR, 10);
-
-        String newFile = "/dir3/file.test";
-        String file1 = "/dir3/file1.test";
-        String file2 = "/dir3/file1.test";
-
-        log.logMakeDirectory(PATH, PRIMARY);
-        log.logRename(PATH, PRIMARY, new IgfsPath(newFile));
-        log.logListDirectory(PATH, PRIMARY, new String[] { file1, file2 });
-        log.logDelete(PATH, PRIMARY, false);
-
-        log.close();
-
-        checkLog(
-            new SB().a(U.jvmPid() + d() + TYPE_DIR_MAKE + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(17)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_RENAME + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(15) + newFile +
-                d(2)).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_DIR_LIST + d() + PATH_STR_ESCAPED + d() + PRIMARY + d(17) + file1 +
-                DELIM_FIELD_VAL + file2).toString(),
-            new SB().a(U.jvmPid() + d() + TYPE_DELETE + d(1) + PATH_STR_ESCAPED + d() + PRIMARY + d(16) + 0 +
-                d()).toString()
-        );
-    }
-
-    /**
-     * Create IGFS file with the given path.
-     *
-     * @param path File path.
-     * @return IGFS file instance.
-     */
-    private IgfsFile file(String path) {
-        return new IgfsFileImpl(new IgfsPath(path), new IgfsFileInfo(), 64 * 1024 * 1024);
-    }
-
-    /**
-     * Ensure that log file has only the following lines.
-     *
-     * @param lines Expected lines.
-     */
-    private void checkLog(String... lines) throws Exception {
-        BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(LOG_FILE)));
-
-        List<String> logLines = new ArrayList<>(lines.length);
-
-        String nextLogLine;
-
-        while ((nextLogLine = br.readLine()) != null)
-            logLines.add(nextLogLine);
-
-        U.closeQuiet(br);
-
-        assertEquals(lines.length + 1, logLines.size());
-
-        assertEquals(logLines.get(0), HDR);
-
-        for (int i = 0; i < lines.length; i++) {
-            String logLine = logLines.get(i + 1);
-
-            logLine = logLine.substring(logLine.indexOf(DELIM_FIELD, logLine.indexOf(DELIM_FIELD) + 1) + 1);
-
-            assertEquals(lines[i], logLine);
-        }
-    }
-
-    /**
-     * Return single field delimiter.
-     *
-     * @return Single field delimiter.
-     */
-    private String d() {
-        return d(1);
-    }
-
-    /**
-     * Return a bunch of field delimiters.
-     *
-     * @param cnt Amount of field delimiters.
-     * @return Field delimiters.
-     */
-    private String d(int cnt) {
-        SB buf = new SB();
-
-        for (int i = 0; i < cnt; i++)
-            buf.a(DELIM_FIELD);
-
-        return buf.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerStateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerStateSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerStateSelfTest.java
deleted file mode 100644
index cbe83f0..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoggerStateSelfTest.java
+++ /dev/null
@@ -1,325 +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.FileSystem;
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.hadoop.v1.*;
-import org.apache.ignite.internal.igfs.common.*;
-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 java.lang.reflect.*;
-import java.net.*;
-import java.nio.file.*;
-import java.util.*;
-
-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.igfs.hadoop.IgfsHadoopParameters.*;
-
-/**
- * Ensures that sampling is really turned on/off.
- */
-public class IgfsHadoopFileSystemLoggerStateSelfTest extends IgfsCommonAbstractTest {
-    /** IGFS. */
-    private IgfsEx igfs;
-
-    /** File system. */
-    private FileSystem fs;
-
-    /** Whether logging is enabled in FS configuration. */
-    private boolean logging;
-
-    /** whether sampling is enabled. */
-    private Boolean sampling;
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        U.closeQuiet(fs);
-
-        igfs = null;
-        fs = null;
-
-        G.stopAll(true);
-
-        logging = false;
-        sampling = null;
-    }
-
-    /**
-     * Startup the grid and instantiate the file system.
-     *
-     * @throws Exception If failed.
-     */
-    private void startUp() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("replicated");
-        igfsCfg.setName("igfs");
-        igfsCfg.setBlockSize(512 * 1024);
-        igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
-            put("type", "tcp");
-            put("port", "10500");
-        }});
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName("partitioned");
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        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("igfs-grid");
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setConnectorConfiguration(null);
-
-        Ignite g = G.start(cfg);
-
-        igfs = (IgfsEx)g.fileSystem("igfs");
-
-        igfs.globalSampling(sampling);
-
-        fs = fileSystem();
-    }
-
-    /**
-     * When logging is disabled and sampling is not set no-op logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingDisabledSamplingNotSet() throws Exception {
-        startUp();
-
-        assert !logEnabled();
-    }
-
-    /**
-     * When logging is enabled and sampling is not set file logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingEnabledSamplingNotSet() throws Exception {
-        logging = true;
-
-        startUp();
-
-        assert logEnabled();
-    }
-
-    /**
-     * When logging is disabled and sampling is disabled no-op logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingDisabledSamplingDisabled() throws Exception {
-        sampling = false;
-
-        startUp();
-
-        assert !logEnabled();
-    }
-
-    /**
-     * When logging is enabled and sampling is disabled no-op logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingEnabledSamplingDisabled() throws Exception {
-        logging = true;
-        sampling = false;
-
-        startUp();
-
-        assert !logEnabled();
-    }
-
-    /**
-     * When logging is disabled and sampling is enabled file logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingDisabledSamplingEnabled() throws Exception {
-        sampling = true;
-
-        startUp();
-
-        assert logEnabled();
-    }
-
-    /**
-     * When logging is enabled and sampling is enabled file logger must be used.
-     *
-     * @throws Exception If failed.
-     */
-    public void testLoggingEnabledSamplingEnabled() throws Exception {
-        logging = true;
-        sampling = true;
-
-        startUp();
-
-        assert logEnabled();
-    }
-
-    /**
-     * Ensure sampling change through API causes changes in logging on subsequent client connections.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSamplingChange() throws Exception {
-        // Start with sampling not set.
-        startUp();
-
-        assert !logEnabled();
-
-        fs.close();
-
-        // "Not set" => true transition.
-        igfs.globalSampling(true);
-
-        fs = fileSystem();
-
-        assert logEnabled();
-
-        fs.close();
-
-        // True => "not set" transition.
-        igfs.globalSampling(null);
-
-        fs = fileSystem();
-
-        assert !logEnabled();
-
-        // "Not-set" => false transition.
-        igfs.globalSampling(false);
-
-        fs = fileSystem();
-
-        assert !logEnabled();
-
-        fs.close();
-
-        // False => "not=set" transition.
-        igfs.globalSampling(null);
-
-        fs = fileSystem();
-
-        assert !logEnabled();
-
-        fs.close();
-
-        // True => false transition.
-        igfs.globalSampling(true);
-        igfs.globalSampling(false);
-
-        fs = fileSystem();
-
-        assert !logEnabled();
-
-        fs.close();
-
-        // False => true transition.
-        igfs.globalSampling(true);
-
-        fs = fileSystem();
-
-        assert logEnabled();
-    }
-
-    /**
-     * Ensure that log directory is set to IGFS when client FS connects.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    public void testLogDirectory() throws Exception {
-        startUp();
-
-        assertEquals(Paths.get(U.getIgniteHome()).normalize().toString(),
-            igfs.clientLogDirectory());
-    }
-
-    /**
-     * Instantiate new file system.
-     *
-     * @return New file system.
-     * @throws Exception If failed.
-     */
-    private IgfsHadoopFileSystem fileSystem() throws Exception {
-        Configuration fsCfg = new Configuration();
-
-        fsCfg.addResource(U.resolveIgniteUrl("modules/core/src/test/config/hadoop/core-site-loopback.xml"));
-
-        fsCfg.setBoolean("fs.igfs.impl.disable.cache", true);
-
-        if (logging)
-            fsCfg.setBoolean(String.format(PARAM_IGFS_LOG_ENABLED, "igfs:igfs-grid@"), logging);
-
-        fsCfg.setStrings(String.format(PARAM_IGFS_LOG_DIR, "igfs:igfs-grid@"), U.getIgniteHome());
-
-        return (IgfsHadoopFileSystem)FileSystem.get(new URI("igfs://igfs:igfs-grid@/"), fsCfg);
-    }
-
-    /**
-     * Ensure that real logger is used by the file system.
-     *
-     * @return {@code True} in case path is secondary.
-     * @throws Exception If failed.
-     */
-    private boolean logEnabled() throws Exception {
-        assert fs != null;
-
-        Field field = fs.getClass().getDeclaredField("clientLog");
-
-        field.setAccessible(true);
-
-        return ((IgfsLogger)field.get(fs)).isLogEnabled();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackAbstractSelfTest.java
deleted file mode 100644
index bd9b031..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackAbstractSelfTest.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.igfs;
-
-import java.util.*;
-
-import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test.
- */
-public abstract class IgfsHadoopFileSystemLoopbackAbstractSelfTest extends
-    IgfsHadoopFileSystemAbstractSelfTest {
-    /**
-     * Constructor.
-     *
-     * @param mode IGFS mode.
-     * @param skipEmbed Skip embedded mode flag.
-     */
-    protected IgfsHadoopFileSystemLoopbackAbstractSelfTest(IgfsMode mode, boolean skipEmbed) {
-        super(mode, skipEmbed, true);
-    }
-
-    /** {@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)));
-        }};
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
deleted file mode 100644
index 6035c7a..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in DUAL_ASYNC mode.
- */
-public class IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest extends
-    IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest() {
-        super(DUAL_ASYNC, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
deleted file mode 100644
index 3900b0b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in DUAL_SYNC mode.
- */
-public class IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest
-    extends IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest() {
-        super(DUAL_SYNC, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
deleted file mode 100644
index dee717e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in PRIMARY mode.
- */
-public class IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest
-    extends IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackEmbeddedPrimarySelfTest() {
-        super(PRIMARY, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
deleted file mode 100644
index 1f13019..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java
+++ /dev/null
@@ -1,34 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in SECONDARY mode.
- */
-public class IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest extends
-    IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackEmbeddedSecondarySelfTest() {
-        super(PROXY, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
deleted file mode 100644
index 4a66da6..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in DUAL_ASYNC mode.
- */
-public class IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest extends
-    IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackExternalDualAsyncSelfTest() {
-        super(DUAL_ASYNC, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
deleted file mode 100644
index f5f49ed..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in DUAL_SYNC mode.
- */
-public class IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest
-    extends IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackExternalDualSyncSelfTest() {
-        super(DUAL_SYNC, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest.java
deleted file mode 100644
index 5903704..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in PRIMARY mode.
- */
-public class IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest
-    extends IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackExternalPrimarySelfTest() {
-        super(PRIMARY, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest.java
deleted file mode 100644
index 0e95f83..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest.java
+++ /dev/null
@@ -1,34 +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.*;
-
-/**
- * IGFS Hadoop file system IPC loopback self test in SECONDARY mode.
- */
-public class IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest extends
-    IgfsHadoopFileSystemLoopbackAbstractSelfTest {
-
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemLoopbackExternalSecondarySelfTest() {
-        super(PROXY, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemSecondaryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemSecondaryModeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemSecondaryModeSelfTest.java
deleted file mode 100644
index b88816a..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemSecondaryModeSelfTest.java
+++ /dev/null
@@ -1,319 +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.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.hadoop.*;
-import org.apache.ignite.igfs.hadoop.v1.*;
-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.lang.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-
-import java.net.*;
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.igfs.IgfsMode.*;
-
-/**
- * Ensures correct modes resolution for SECONDARY paths.
- */
-public class IgfsHadoopFileSystemSecondaryModeSelfTest extends IgfsCommonAbstractTest {
-    /** Path to check. */
-    private static final Path PATH = new Path("/dir");
-
-    /** Pattern matching the path. */
-    private static final String PATTERN_MATCHES = "/dir";
-
-    /** Pattern doesn't matching the path. */
-    private static final String PATTERN_NOT_MATCHES = "/files";
-
-    /** Default IGFS mode. */
-    private IgfsMode mode;
-
-    /** Path modes. */
-    private Map<String, IgfsMode> pathModes;
-
-    /** File system. */
-    private IgfsHadoopFileSystem fs;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        mode = null;
-        pathModes = null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        U.closeQuiet(fs);
-
-        fs = null;
-
-        G.stopAll(true);
-    }
-
-    /**
-     * Perform initial startup.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("NullableProblems")
-    private void startUp() throws Exception {
-        startUpSecondary();
-
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("replicated");
-        igfsCfg.setName("igfs");
-        igfsCfg.setBlockSize(512 * 1024);
-        igfsCfg.setDefaultMode(mode);
-        igfsCfg.setPathModes(pathModes);
-        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
-            put("type", "tcp");
-            put("port", "10500");
-        }});
-
-        igfsCfg.setManagementPort(-1);
-        igfsCfg.setSecondaryFileSystem(new IgfsHadoopFileSystemWrapper(
-            "igfs://igfs-secondary:igfs-grid-secondary@127.0.0.1:11500/",
-            "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml"));
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName("partitioned");
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        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("igfs-grid");
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-
-        G.start(cfg);
-
-        Configuration fsCfg = new Configuration();
-
-        fsCfg.addResource(U.resolveIgniteUrl("modules/core/src/test/config/hadoop/core-site-loopback.xml"));
-
-        fsCfg.setBoolean("fs.igfs.impl.disable.cache", true);
-
-        fs = (IgfsHadoopFileSystem)FileSystem.get(new URI("igfs://igfs:igfs-grid@/"), fsCfg);
-    }
-
-    /**
-     * Startup secondary file system.
-     *
-     * @throws Exception If failed.
-     */
-    private void startUpSecondary() throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("replicated");
-        igfsCfg.setName("igfs-secondary");
-        igfsCfg.setBlockSize(512 * 1024);
-        igfsCfg.setDefaultMode(PRIMARY);
-        igfsCfg.setIpcEndpointConfiguration(new HashMap<String, String>() {{
-            put("type", "tcp");
-            put("port", "11500");
-        }});
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName("partitioned");
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
-        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("igfs-grid-secondary");
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-
-        G.start(cfg);
-    }
-
-    /**
-     * Check path resolution when secondary mode is not default and there are no other exclusion paths.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryNotDefaultNoExclusions() throws Exception {
-        mode = PRIMARY;
-
-        startUp();
-
-        assert !secondary(PATH);
-        assert !secondary(PATH);
-    }
-
-    /**
-     * Check path resolution when secondary mode is not default and there is no matching exclusion paths.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryNotDefaultNonMatchingExclusion() throws Exception {
-        mode = PRIMARY;
-
-        pathModes(F.t(PATTERN_NOT_MATCHES, PROXY));
-
-        startUp();
-
-        assert !secondary(PATH);
-        assert !secondary(PATH);
-    }
-
-    /**
-     * Check path resolution when secondary mode is not default and there is matching exclusion path.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryNotDefaultMatchingExclusion() throws Exception {
-        mode = PRIMARY;
-
-        pathModes(F.t(PATTERN_NOT_MATCHES, PROXY), F.t(PATTERN_MATCHES, PROXY));
-
-        startUp();
-
-        assert secondary(PATH);
-        assert secondary(PATH);
-    }
-
-    /**
-     * Check path resolution when secondary mode is default and there is no exclusion paths.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryDefaultNoExclusions() throws Exception {
-        mode = PROXY;
-
-        startUp();
-
-        assert secondary(PATH);
-        assert secondary(PATH);
-    }
-
-    /**
-     * Check path resolution when secondary mode is default and there is no matching exclusion paths.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryDefaultNonMatchingExclusion() throws Exception {
-        mode = PROXY;
-
-        pathModes(F.t(PATTERN_NOT_MATCHES, PRIMARY));
-
-        startUp();
-
-        assert secondary(PATH);
-        assert secondary(PATH);
-    }
-
-    /**
-     * Check path resolution when secondary mode is default and there is no matching exclusion paths.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSecondaryDefaultMatchingExclusion() throws Exception {
-        mode = PROXY;
-
-        pathModes(F.t(PATTERN_NOT_MATCHES, PRIMARY), F.t(PATTERN_MATCHES, PRIMARY));
-
-        startUp();
-
-        assert !secondary(PATH);
-        assert !secondary(PATH);
-    }
-
-    /**
-     * Set IGFS modes for particular paths.
-     *
-     * @param modes Modes.
-     */
-    @SafeVarargs
-    final void pathModes(IgniteBiTuple<String, IgfsMode>... modes) {
-        assert modes != null;
-
-        pathModes = new LinkedHashMap<>(modes.length, 1.0f);
-
-        for (IgniteBiTuple<String, IgfsMode> mode : modes)
-            pathModes.put(mode.getKey(), mode.getValue());
-    }
-
-    /**
-     * Check whether the given path is threaten as SECONDARY in the file system.
-     *
-     * @param path Path to check.
-     * @return {@code True} in case path is secondary.
-     * @throws Exception If failed.
-     */
-    private boolean secondary(Path path) throws Exception {
-        return fs.mode(path) == PROXY;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemAbstractSelfTest.java
deleted file mode 100644
index ed7b7cc..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemAbstractSelfTest.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.igfs;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.ipc.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.testframework.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
-
-/**
- * IGFS Hadoop file system IPC self test.
- */
-public abstract class IgfsHadoopFileSystemShmemAbstractSelfTest extends IgfsHadoopFileSystemAbstractSelfTest {
-    /**
-     * Constructor.
-     *
-     * @param mode IGFS mode.
-     * @param skipEmbed Skip embedded mode flag.
-     */
-    protected IgfsHadoopFileSystemShmemAbstractSelfTest(IgfsMode mode, boolean skipEmbed) {
-        super(mode, skipEmbed, false);
-    }
-
-    /** {@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)));
-        }};
-    }
-
-    /**
-     * Checks correct behaviour in case when we run out of system
-     * resources.
-     *
-     * @throws Exception If error occurred.
-     */
-    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testOutOfResources() throws Exception {
-        final Collection<IpcEndpoint> eps = new LinkedList<>();
-
-        try {
-            IgniteCheckedException e = (IgniteCheckedException)GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @SuppressWarnings("InfiniteLoopStatement")
-                @Override public Object call() throws Exception {
-                    while (true) {
-                        IpcEndpoint ep = IpcEndpointFactory.connectEndpoint("shmem:10500", log);
-
-                        eps.add(ep);
-                    }
-                }
-            }, IgniteCheckedException.class, null);
-
-            assertNotNull(e);
-
-            String msg = e.getMessage();
-
-            assertTrue("Invalid exception: " + X.getFullStackTrace(e),
-                msg.contains("(error code: 28)") ||
-                msg.contains("(error code: 24)") ||
-                msg.contains("(error code: 12)"));
-        }
-        finally {
-            for (IpcEndpoint ep : eps)
-                ep.close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
deleted file mode 100644
index d11e8d1..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in DUAL_ASYNC mode.
- */
-public class IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemEmbeddedDualAsyncSelfTest() {
-        super(DUAL_ASYNC, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
deleted file mode 100644
index 6138215..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in DUAL_SYNC mode.
- */
-public class IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemEmbeddedDualSyncSelfTest() {
-        super(DUAL_SYNC, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
deleted file mode 100644
index 9fc2d33..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in PRIMARY mode.
- */
-public class IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemEmbeddedPrimarySelfTest() {
-        super(PRIMARY, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
deleted file mode 100644
index a05158c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in SECONDARY mode.
- */
-public class IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemEmbeddedSecondarySelfTest() {
-        super(PROXY, false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest.java
deleted file mode 100644
index c132e5b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in DUAL_ASYNC mode.
- */
-public class IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemExternalDualAsyncSelfTest() {
-        super(DUAL_ASYNC, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualSyncSelfTest.java
deleted file mode 100644
index 21b393d..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalDualSyncSelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in DUAL_SYNC mode.
- */
-public class IgfsHadoopFileSystemShmemExternalDualSyncSelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemExternalDualSyncSelfTest() {
-        super(DUAL_SYNC, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalPrimarySelfTest.java
deleted file mode 100644
index f147929..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalPrimarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in PRIMARY mode.
- */
-public class IgfsHadoopFileSystemShmemExternalPrimarySelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemExternalPrimarySelfTest() {
-        super(PRIMARY, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalSecondarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalSecondarySelfTest.java
deleted file mode 100644
index 7c77740..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopFileSystemShmemExternalSecondarySelfTest.java
+++ /dev/null
@@ -1,33 +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.*;
-
-/**
- * IGFS Hadoop file system IPC shmem self test in SECONDARY mode.
- */
-public class IgfsHadoopFileSystemShmemExternalSecondarySelfTest
-    extends IgfsHadoopFileSystemShmemAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public IgfsHadoopFileSystemShmemExternalSecondarySelfTest() {
-        super(PROXY, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
index d27d93d..d128731 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
@@ -75,7 +75,7 @@ public class IgfsNearOnlyMultiNodeSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("partitioned");
         igfsCfg.setMetaCacheName("partitioned");
@@ -88,7 +88,7 @@ public class IgfsNearOnlyMultiNodeSelfTest extends GridCommonAbstractTest {
 
         igfsCfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setCacheConfiguration(cacheConfiguration(gridName));
 


[30/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
new file mode 100644
index 0000000..a8eb58c
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java
new file mode 100644
index 0000000..c335a62
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java
new file mode 100644
index 0000000..88dd896
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
new file mode 100644
index 0000000..c9c61fe
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java
new file mode 100644
index 0000000..c2dad82
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+/**
+ * Listens to the events of {@link HadoopIgfsIpcIo}.
+ */
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java
new file mode 100644
index 0000000..0c29454
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.igfs;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * JCL logger wrapper for Hadoop.
+ */
+public class HadoopIgfsJclLogger implements IgniteLogger {
+    /** JCL implementation proxy. */
+    @GridToStringInclude
+    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 S.toString(HadoopIgfsJclLogger.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
new file mode 100644
index 0000000..662541a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java
new file mode 100644
index 0000000..902d710
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/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.processors.hadoop.igfs;
+
+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;
+    }
+}


[36/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/IgfsHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/IgfsHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/IgfsHadoopFileSystem.java
deleted file mode 100644
index 0759203..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/IgfsHadoopFileSystem.java
+++ /dev/null
@@ -1,982 +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.hadoop.v2;
-
-import org.apache.commons.logging.*;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-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.net.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-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.igfs.hadoop.IgfsHadoopParameters.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
-import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
-
-/**
- * {@code IGFS} Hadoop 2.x file system driver over file system API. To use
- * {@code IGFS} as Hadoop file system, you should configure this class
- * in Hadoop's {@code core-site.xml} as follows:
- * <pre name="code" class="xml">
- *  &lt;property&gt;
- *      &lt;name&gt;fs.default.name&lt;/name&gt;
- *      &lt;value&gt;igfs://ipc&lt;/value&gt;
- *  &lt;/property&gt;
- *
- *  &lt;property&gt;
- *      &lt;name&gt;fs.igfs.impl&lt;/name&gt;
- *      &lt;value&gt;org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem&lt;/value&gt;
- *  &lt;/property&gt;
- * </pre>
- * You should also add Ignite JAR and all libraries to Hadoop classpath. To
- * do this, add following lines to {@code conf/hadoop-env.sh} script in Hadoop
- * distribution:
- * <pre name="code" class="bash">
- * export IGNITE_HOME=/path/to/Ignite/distribution
- * export HADOOP_CLASSPATH=$IGNITE_HOME/ignite*.jar
- *
- * for f in $IGNITE_HOME/libs/*.jar; do
- *  export HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$f;
- * done
- * </pre>
- * <h1 class="header">Data vs Clients Nodes</h1>
- * Hadoop needs to use its FileSystem remotely from client nodes as well as directly on
- * data nodes. Client nodes are responsible for basic file system operations as well as
- * accessing data nodes remotely. Usually, client nodes are started together
- * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
- * started together with Hadoop {@code task-tracker} processes.
- * <p>
- * For sample client and data node configuration refer to {@code config/hadoop/default-config-client.xml}
- * and {@code config/hadoop/default-config.xml} configuration files in Ignite installation.
- */
-public class IgfsHadoopFileSystem extends AbstractFileSystem implements Closeable {
-    /** Logger. */
-    private static final Log LOG = LogFactory.getLog(IgfsHadoopFileSystem.class);
-
-    /** Ensures that close routine is invoked at most once. */
-    private final AtomicBoolean closeGuard = new AtomicBoolean();
-
-    /** Grid remote client. */
-    private IgfsHadoopWrapper rmtClient;
-
-    /** Working directory. */
-    private IgfsPath workingDir;
-
-    /** URI. */
-    private URI uri;
-
-    /** Authority. */
-    private String uriAuthority;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Server block size. */
-    private long grpBlockSize;
-
-    /** Default replication factor. */
-    private short dfltReplication;
-
-    /** Secondary URI string. */
-    private URI secondaryUri;
-
-    /** Mode resolver. */
-    private IgfsModeResolver modeRslvr;
-
-    /** Secondary file system instance. */
-    private AbstractFileSystem secondaryFs;
-
-    /** Whether custom sequential reads before prefetch value is provided. */
-    private boolean seqReadsBeforePrefetchOverride;
-
-    /** Custom-provided sequential reads before prefetch. */
-    private int seqReadsBeforePrefetch;
-
-    /** Flag that controls whether file writes should be colocated on data node. */
-    private boolean colocateFileWrites;
-
-    /** Prefer local writes. */
-    private boolean preferLocFileWrites;
-
-    /**
-     * @param name URI for file system.
-     * @param cfg Configuration.
-     * @throws URISyntaxException if name has invalid syntax.
-     * @throws IOException If initialization failed.
-     */
-    public IgfsHadoopFileSystem(URI name, Configuration cfg) throws URISyntaxException, IOException {
-        super(IgfsHadoopEndpoint.normalize(name), IGFS_SCHEME, false, -1);
-
-        uri = name;
-
-        try {
-            initialize(name, cfg);
-        }
-        catch (IOException e) {
-            // Close client if exception occurred.
-            if (rmtClient != null)
-                rmtClient.close(false);
-
-            throw e;
-        }
-
-        workingDir = new IgfsPath("/user/" + cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void checkPath(Path path) {
-        URI uri = path.toUri();
-
-        if (uri.isAbsolute()) {
-            if (!F.eq(uri.getScheme(), IGFS_SCHEME))
-                throw new InvalidPathException("Wrong path scheme [expected=" + IGFS_SCHEME + ", actual=" +
-                    uri.getAuthority() + ']');
-
-            if (!F.eq(uri.getAuthority(), uriAuthority))
-                throw new InvalidPathException("Wrong path authority [expected=" + uriAuthority + ", actual=" +
-                    uri.getAuthority() + ']');
-        }
-    }
-
-    /**
-     * Public setter that can be used by direct users of FS or Visor.
-     *
-     * @param colocateFileWrites Whether all ongoing file writes should be colocated.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public void colocateFileWrites(boolean colocateFileWrites) {
-        this.colocateFileWrites = colocateFileWrites;
-    }
-
-    /**
-     * Enter busy state.
-     *
-     * @throws IOException If file system is stopped.
-     */
-    private void enterBusy() throws IOException {
-        if (closeGuard.get())
-            throw new IOException("File system is stopped.");
-    }
-
-    /**
-     * Leave busy state.
-     */
-    private void leaveBusy() {
-        // No-op.
-    }
-
-    /**
-     * @param name URI passed to constructor.
-     * @param cfg Configuration passed to constructor.
-     * @throws IOException If initialization failed.
-     */
-    private void initialize(URI name, Configuration cfg) throws IOException {
-        enterBusy();
-
-        try {
-            if (rmtClient != null)
-                throw new IOException("File system is already initialized: " + rmtClient);
-
-            A.notNull(name, "name");
-            A.notNull(cfg, "cfg");
-
-            if (!IGFS_SCHEME.equals(name.getScheme()))
-                throw new IOException("Illegal file system URI [expected=" + IGFS_SCHEME +
-                    "://[name]/[optional_path], actual=" + name + ']');
-
-            uriAuthority = name.getAuthority();
-
-            // Override sequential reads before prefetch if needed.
-            seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
-
-            if (seqReadsBeforePrefetch > 0)
-                seqReadsBeforePrefetchOverride = true;
-
-            // In Ignite replication factor is controlled by data cache affinity.
-            // We use replication factor to force the whole file to be stored on local node.
-            dfltReplication = (short)cfg.getInt("dfs.replication", 3);
-
-            // Get file colocation control flag.
-            colocateFileWrites = parameter(cfg, PARAM_IGFS_COLOCATED_WRITES, uriAuthority, false);
-            preferLocFileWrites = cfg.getBoolean(PARAM_IGFS_PREFER_LOCAL_WRITES, false);
-
-            // Get log directory.
-            String logDirCfg = parameter(cfg, PARAM_IGFS_LOG_DIR, uriAuthority, DFLT_IGFS_LOG_DIR);
-
-            File logDirFile = U.resolveIgnitePath(logDirCfg);
-
-            String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
-
-            rmtClient = new IgfsHadoopWrapper(uriAuthority, logDir, cfg, LOG);
-
-            // Handshake.
-            IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
-
-            grpBlockSize = handshake.blockSize();
-
-            IgfsPaths paths = handshake.secondaryPaths();
-
-            Boolean logEnabled = parameter(cfg, PARAM_IGFS_LOG_ENABLED, uriAuthority, false);
-
-            if (handshake.sampling() != null ? handshake.sampling() : logEnabled) {
-                // Initiate client logger.
-                if (logDir == null)
-                    throw new IOException("Failed to resolve log directory: " + logDirCfg);
-
-                Integer batchSize = parameter(cfg, PARAM_IGFS_LOG_BATCH_SIZE, uriAuthority, DFLT_IGFS_LOG_BATCH_SIZE);
-
-                clientLog = IgfsLogger.logger(uriAuthority, handshake.igfsName(), logDir, batchSize);
-            }
-            else
-                clientLog = IgfsLogger.disabledLogger();
-
-            modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes());
-
-            boolean initSecondary = paths.defaultMode() == PROXY;
-
-            if (!initSecondary && paths.pathModes() != null) {
-                for (T2<IgfsPath, IgfsMode> pathMode : paths.pathModes()) {
-                    IgfsMode mode = pathMode.getValue();
-
-                    if (mode == PROXY) {
-                        initSecondary = true;
-
-                        break;
-                    }
-                }
-            }
-
-            if (initSecondary) {
-                Map<String, String> props = paths.properties();
-
-                String secUri = props.get(SECONDARY_FS_URI);
-                String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-
-                try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
-
-                    secondaryFs = secProvider.createAbstractFileSystem();
-                    secondaryUri = secProvider.uri();
-                }
-                catch (IOException e) {
-                    throw new IOException("Failed to connect to the secondary file system: " + secUri, e);
-                }
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        if (closeGuard.compareAndSet(false, true)) {
-            if (rmtClient == null)
-                return;
-
-            rmtClient.close(false);
-
-            if (clientLog.isLogEnabled())
-                clientLog.close();
-
-            // Reset initialized resources.
-            rmtClient = null;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public URI getUri() {
-        return uri;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getUriDefaultPort() {
-        return -1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FsServerDefaults getServerDefaults() throws IOException {
-        return new FsServerDefaults(grpBlockSize, (int)grpBlockSize, (int)grpBlockSize, dfltReplication, 64 * 1024,
-            false, 0, DataChecksum.Type.NULL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean setReplication(Path f, short replication) throws IOException {
-        return mode(f) == PROXY && secondaryFs.setReplication(f, replication);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setTimes(Path f, long mtime, long atime) throws IOException {
-        if (mode(f) == PROXY)
-            secondaryFs.setTimes(f, mtime, atime);
-        else {
-            if (mtime == -1 && atime == -1)
-                return;
-
-            rmtClient.setTimes(convert(f), atime, mtime);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FsStatus getFsStatus() throws IOException {
-        IgfsStatus status = rmtClient.fsStatus();
-
-        return new FsStatus(status.spaceTotal(), status.spaceUsed(), status.spaceTotal() - status.spaceUsed());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setPermission(Path p, FsPermission perm) throws IOException {
-        enterBusy();
-
-        try {
-            A.notNull(p, "p");
-
-            if (mode(p) == PROXY)
-                secondaryFs.setPermission(toSecondary(p), perm);
-            else {
-                if (rmtClient.update(convert(p), permission(perm)) == null)
-                    throw new IOException("Failed to set file permission (file not found?)" +
-                        " [path=" + p + ", perm=" + perm + ']');
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setOwner(Path p, String usr, String grp) throws IOException {
-        A.notNull(p, "p");
-        A.notNull(usr, "username");
-        A.notNull(grp, "grpName");
-
-        enterBusy();
-
-        try {
-            if (mode(p) == PROXY)
-                secondaryFs.setOwner(toSecondary(p), usr, grp);
-            else if (rmtClient.update(convert(p), F.asMap(PROP_USER_NAME, usr, PROP_GROUP_NAME, grp)) == null)
-                throw new IOException("Failed to set file permission (file not found?)" +
-                    " [path=" + p + ", username=" + usr + ", grpName=" + grp + ']');
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FSDataInputStream open(Path f, int bufSize) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = modeRslvr.resolveMode(path);
-
-            if (mode == PROXY) {
-                FSDataInputStream is = secondaryFs.open(toSecondary(f), bufSize);
-
-                if (clientLog.isLogEnabled()) {
-                    // At this point we do not know file size, so we perform additional request to remote FS to get it.
-                    FileStatus status = secondaryFs.getFileStatus(toSecondary(f));
-
-                    long size = status != null ? status.getLen() : -1;
-
-                    long logId = IgfsLogger.nextId();
-
-                    clientLog.logOpen(logId, path, PROXY, bufSize, size);
-
-                    return new FSDataInputStream(new IgfsHadoopProxyInputStream(is, clientLog, logId));
-                }
-                else
-                    return is;
-            }
-            else {
-                IgfsHadoopStreamDelegate stream = seqReadsBeforePrefetchOverride ?
-                    rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
-
-                long logId = -1;
-
-                if (clientLog.isLogEnabled()) {
-                    logId = IgfsLogger.nextId();
-
-                    clientLog.logOpen(logId, path, mode, bufSize, stream.length());
-                }
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
-                        ", bufSize=" + bufSize + ']');
-
-                IgfsHadoopInputStream igfsIn = new IgfsHadoopInputStream(stream, stream.length(),
-                    bufSize, LOG, clientLog, logId);
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Opened input stream [path=" + path + ", delegate=" + stream + ']');
-
-                return new FSDataInputStream(igfsIn);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public FSDataOutputStream createInternal(
-        Path f,
-        EnumSet<CreateFlag> flag,
-        FsPermission perm,
-        int bufSize,
-        short replication,
-        long blockSize,
-        Progressable progress,
-        Options.ChecksumOpt checksumOpt,
-        boolean createParent
-    ) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
-        boolean append = flag.contains(CreateFlag.APPEND);
-        boolean create = flag.contains(CreateFlag.CREATE);
-
-        OutputStream out = null;
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = modeRslvr.resolveMode(path);
-
-            if (LOG.isDebugEnabled())
-                LOG.debug("Opening output stream in create [thread=" + Thread.currentThread().getName() + "path=" +
-                    path + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ']');
-
-            if (mode == PROXY) {
-                FSDataOutputStream os = secondaryFs.createInternal(toSecondary(f), flag, perm, bufSize,
-                    replication, blockSize, progress, checksumOpt, createParent);
-
-                if (clientLog.isLogEnabled()) {
-                    long logId = IgfsLogger.nextId();
-
-                    if (append)
-                        clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
-                    else
-                        clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
-
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(os, clientLog, logId));
-                }
-                else
-                    return os;
-            }
-            else {
-                Map<String, String> permMap = F.asMap(PROP_PERMISSION, toString(perm),
-                    PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites));
-
-                // Create stream and close it in the 'finally' section if any sequential operation failed.
-                IgfsHadoopStreamDelegate stream;
-
-                long logId = -1;
-
-                if (append) {
-                    stream = rmtClient.append(path, create, permMap);
-
-                    if (clientLog.isLogEnabled()) {
-                        logId = IgfsLogger.nextId();
-
-                        clientLog.logAppend(logId, path, mode, bufSize);
-                    }
-
-                    if (LOG.isDebugEnabled())
-                        LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
-                }
-                else {
-                    stream = rmtClient.create(path, overwrite, colocateFileWrites, replication, blockSize,
-                        permMap);
-
-                    if (clientLog.isLogEnabled()) {
-                        logId = IgfsLogger.nextId();
-
-                        clientLog.logCreate(logId, path, mode, overwrite, bufSize, replication, blockSize);
-                    }
-
-                    if (LOG.isDebugEnabled())
-                        LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
-                }
-
-                assert stream != null;
-
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG,
-                    clientLog, logId);
-
-                bufSize = Math.max(64 * 1024, bufSize);
-
-                out = new BufferedOutputStream(igfsOut, bufSize);
-
-                FSDataOutputStream res = new FSDataOutputStream(out, null, 0);
-
-                // Mark stream created successfully.
-                out = null;
-
-                return res;
-            }
-        }
-        finally {
-            // Close if failed during stream creation.
-            if (out != null)
-                U.closeQuiet(out);
-
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean supportsSymlinks() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void renameInternal(Path src, Path dst) throws IOException {
-        A.notNull(src, "src");
-        A.notNull(dst, "dst");
-
-        enterBusy();
-
-        try {
-            IgfsPath srcPath = convert(src);
-            IgfsPath dstPath = convert(dst);
-            Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(srcPath);
-
-            if (childrenModes.contains(PROXY)) {
-                if (clientLog.isLogEnabled())
-                    clientLog.logRename(srcPath, PROXY, dstPath);
-
-                secondaryFs.renameInternal(toSecondary(src), toSecondary(dst));
-            }
-
-            rmtClient.rename(srcPath, dstPath);
-
-            if (clientLog.isLogEnabled())
-                clientLog.logRename(srcPath, modeRslvr.resolveMode(srcPath), dstPath);
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(Path f, boolean recursive) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = modeRslvr.resolveMode(path);
-            Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
-
-            if (childrenModes.contains(PROXY)) {
-                if (clientLog.isLogEnabled())
-                    clientLog.logDelete(path, PROXY, recursive);
-
-                return secondaryFs.delete(toSecondary(f), recursive);
-            }
-
-            boolean res = rmtClient.delete(path, recursive);
-
-            if (clientLog.isLogEnabled())
-                clientLog.logDelete(path, mode, recursive);
-
-            return res;
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setVerifyChecksum(boolean verifyChecksum) throws IOException {
-        // Checksum has effect for secondary FS only.
-        if (secondaryFs != null)
-            secondaryFs.setVerifyChecksum(verifyChecksum);
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileChecksum getFileChecksum(Path f) throws IOException {
-        if (mode(f) == PROXY)
-            return secondaryFs.getFileChecksum(f);
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus[] listStatus(Path f) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = modeRslvr.resolveMode(path);
-
-            if (mode == PROXY) {
-                FileStatus[] arr = secondaryFs.listStatus(toSecondary(f));
-
-                if (arr == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = toPrimary(arr[i]);
-
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
-
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, PROXY, fileArr);
-                }
-
-                return arr;
-            }
-            else {
-                Collection<IgfsFile> list = rmtClient.listFiles(path);
-
-                if (list == null)
-                    throw new FileNotFoundException("File " + f + " does not exist.");
-
-                List<IgfsFile> files = new ArrayList<>(list);
-
-                FileStatus[] arr = new FileStatus[files.size()];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(files.get(i));
-
-                if (clientLog.isLogEnabled()) {
-                    String[] fileArr = new String[arr.length];
-
-                    for (int i = 0; i < arr.length; i++)
-                        fileArr[i] = arr[i].getPath().toString();
-
-                    clientLog.logListDirectory(path, mode, fileArr);
-                }
-
-                return arr;
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdir(Path f, FsPermission perm, boolean createParent) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            IgfsPath path = convert(f);
-            IgfsMode mode = modeRslvr.resolveMode(path);
-
-            if (mode == PROXY) {
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, PROXY);
-
-                secondaryFs.mkdir(toSecondary(f), perm, createParent);
-            }
-            else {
-                rmtClient.mkdirs(path, permission(perm));
-
-                if (clientLog.isLogEnabled())
-                    clientLog.logMakeDirectory(path, mode);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus getFileStatus(Path f) throws IOException {
-        A.notNull(f, "f");
-
-        enterBusy();
-
-        try {
-            if (mode(f) == PROXY)
-                return toPrimary(secondaryFs.getFileStatus(toSecondary(f)));
-            else {
-                IgfsFile info = rmtClient.info(convert(f));
-
-                if (info == null)
-                    throw new FileNotFoundException("File not found: " + f);
-
-                return convert(info);
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public BlockLocation[] getFileBlockLocations(Path path, long start, long len) throws IOException {
-        A.notNull(path, "path");
-
-        IgfsPath igfsPath = convert(path);
-
-        enterBusy();
-
-        try {
-            if (modeRslvr.resolveMode(igfsPath) == PROXY)
-                return secondaryFs.getFileBlockLocations(path, start, len);
-            else {
-                long now = System.currentTimeMillis();
-
-                List<IgfsBlockLocation> affinity = new ArrayList<>(
-                    rmtClient.affinity(igfsPath, start, len));
-
-                BlockLocation[] arr = new BlockLocation[affinity.size()];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = convert(affinity.get(i));
-
-                if (LOG.isDebugEnabled())
-                    LOG.debug("Fetched file locations [path=" + path + ", fetchTime=" +
-                        (System.currentTimeMillis() - now) + ", locations=" + Arrays.asList(arr) + ']');
-
-                return arr;
-            }
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /**
-     * Resolve path mode.
-     *
-     * @param path HDFS path.
-     * @return Path mode.
-     */
-    public IgfsMode mode(Path path) {
-        return modeRslvr.resolveMode(convert(path));
-    }
-
-    /**
-     * Convert the given path to path acceptable by the primary file system.
-     *
-     * @param path Path.
-     * @return Primary file system path.
-     */
-    private Path toPrimary(Path path) {
-        return convertPath(path, getUri());
-    }
-
-    /**
-     * Convert the given path to path acceptable by the secondary file system.
-     *
-     * @param path Path.
-     * @return Secondary file system path.
-     */
-    private Path toSecondary(Path path) {
-        assert secondaryFs != null;
-        assert secondaryUri != null;
-
-        return convertPath(path, secondaryUri);
-    }
-
-    /**
-     * Convert path using the given new URI.
-     *
-     * @param path Old path.
-     * @param newUri New URI.
-     * @return New path.
-     */
-    private Path convertPath(Path path, URI newUri) {
-        assert newUri != null;
-
-        if (path != null) {
-            URI pathUri = path.toUri();
-
-            try {
-                return new Path(new URI(pathUri.getScheme() != null ? newUri.getScheme() : null,
-                    pathUri.getAuthority() != null ? newUri.getAuthority() : null, pathUri.getPath(), null, null));
-            }
-            catch (URISyntaxException e) {
-                throw new IgniteException("Failed to construct secondary file system path from the primary file " +
-                    "system path: " + path, e);
-            }
-        }
-        else
-            return null;
-    }
-
-    /**
-     * Convert a file status obtained from the secondary file system to a status of the primary file system.
-     *
-     * @param status Secondary file system status.
-     * @return Primary file system status.
-     */
-    private FileStatus toPrimary(FileStatus status) {
-        return status != null ? new FileStatus(status.getLen(), status.isDirectory(), status.getReplication(),
-            status.getBlockSize(), status.getModificationTime(), status.getAccessTime(), status.getPermission(),
-            status.getOwner(), status.getGroup(), toPrimary(status.getPath())) : null;
-    }
-
-    /**
-     * Convert IGFS path into Hadoop path.
-     *
-     * @param path IGFS path.
-     * @return Hadoop path.
-     */
-    private Path convert(IgfsPath path) {
-        return new Path(IGFS_SCHEME, uriAuthority, path.toString());
-    }
-
-    /**
-     * Convert Hadoop path into IGFS path.
-     *
-     * @param path Hadoop path.
-     * @return IGFS path.
-     */
-    @Nullable private IgfsPath convert(Path path) {
-        if (path == null)
-            return null;
-
-        return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
-            new IgfsPath(workingDir, path.toUri().getPath());
-    }
-
-    /**
-     * Convert IGFS affinity block location into Hadoop affinity block location.
-     *
-     * @param block IGFS affinity block location.
-     * @return Hadoop affinity block location.
-     */
-    private BlockLocation convert(IgfsBlockLocation block) {
-        Collection<String> names = block.names();
-        Collection<String> hosts = block.hosts();
-
-        return new BlockLocation(
-            names.toArray(new String[names.size()]) /* hostname:portNumber of data nodes */,
-            hosts.toArray(new String[hosts.size()]) /* hostnames of data nodes */,
-            block.start(), block.length()
-        ) {
-            @Override public String toString() {
-                try {
-                    return "BlockLocation [offset=" + getOffset() + ", length=" + getLength() +
-                        ", hosts=" + Arrays.asList(getHosts()) + ", names=" + Arrays.asList(getNames()) + ']';
-                }
-                catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Convert IGFS file information into Hadoop file status.
-     *
-     * @param file IGFS file information.
-     * @return Hadoop file status.
-     */
-    private FileStatus convert(IgfsFile file) {
-        return new FileStatus(
-            file.length(),
-            file.isDirectory(),
-            dfltReplication,
-            file.groupBlockSize(),
-            file.modificationTime(),
-            file.accessTime(),
-            permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME),
-            file.property(PROP_GROUP_NAME, "users"),
-            convert(file.path())) {
-            @Override public String toString() {
-                return "FileStatus [path=" + getPath() + ", isDir=" + isDirectory() + ", len=" + getLen() + "]";
-            }
-        };
-    }
-
-    /**
-     * Convert Hadoop permission into IGFS file attribute.
-     *
-     * @param perm Hadoop permission.
-     * @return IGFS attributes.
-     */
-    private Map<String, String> permission(FsPermission perm) {
-        if (perm == null)
-            perm = FsPermission.getDefault();
-
-        return F.asMap(PROP_PERMISSION, toString(perm));
-    }
-
-    /**
-     * @param perm Permission.
-     * @return String.
-     */
-    private static String toString(FsPermission perm) {
-        return String.format("%04o", perm.toShort());
-    }
-
-    /**
-     * Convert IGFS file attributes into Hadoop permission.
-     *
-     * @param file File info.
-     * @return Hadoop permission.
-     */
-    private FsPermission permission(IgfsFile file) {
-        String perm = file.property(PROP_PERMISSION, null);
-
-        if (perm == null)
-            return FsPermission.getDefault();
-
-        try {
-            return new FsPermission((short)Integer.parseInt(perm, 8));
-        }
-        catch (NumberFormatException ignore) {
-            return FsPermission.getDefault();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsHadoopFileSystem.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/package.html b/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/package.html
deleted file mode 100644
index 6df66f4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/igfs/hadoop/v2/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 Hadoop 2.x <code>FileSystem</code> wrapper for Ignite file system.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/main/java/org/apache/ignite/igfs/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/igfs/package.html b/modules/hadoop/src/main/java/org/apache/ignite/igfs/package.html
deleted file mode 100644
index ec38a21..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/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 <b>IG</b>nite <b>F</b>ile <b>S</b>ystem APIs.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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 + ']');
-    }
-}


[25/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..422d941
--- /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<HadoopJobId, HadoopShuffleJob<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, (HadoopMessage)msg);
+                }
+            });
+    }
+
+    /**
+     * Stops shuffle.
+     *
+     * @param cancel If should cancel all ongoing activities.
+     */
+    @Override public void stop(boolean cancel) {
+        for (HadoopShuffleJob 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 HadoopShuffleJob<UUID> newJob(HadoopJobId jobId) throws IgniteCheckedException {
+        HadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
+
+        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()];
+
+        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 HadoopShuffleJob<UUID> job(HadoopJobId jobId) throws IgniteCheckedException {
+        HadoopShuffleJob<UUID> res = jobs.get(jobId);
+
+        if (res == null) {
+            res = newJob(jobId);
+
+            HadoopShuffleJob<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(HadoopShuffleJob<UUID> shuffleJob) {
+        shuffleJob.startSending(ctx.kernalContext().gridName(),
+            new IgniteInClosure2X<UUID, HadoopShuffleMessage>() {
+                @Override public void applyx(UUID dest, HadoopShuffleMessage 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, HadoopMessage msg) {
+        if (msg instanceof HadoopShuffleMessage) {
+            HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
+
+            try {
+                job(m.jobId()).onShuffleMessage(m);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Message handling failed.", e);
+            }
+
+            try {
+                // Reply with ack.
+                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 HadoopShuffleAck) {
+            HadoopShuffleAck m = (HadoopShuffleAck)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 HadoopTaskOutput output(HadoopTaskContext taskCtx) throws IgniteCheckedException {
+        return job(taskCtx.taskInfo().jobId()).output(taskCtx);
+    }
+
+    /**
+     * @param taskCtx Task info.
+     * @return Input.
+     */
+    public HadoopTaskInput input(HadoopTaskContext taskCtx) throws IgniteCheckedException {
+        return job(taskCtx.taskInfo().jobId()).input(taskCtx);
+    }
+
+    /**
+     * @param jobId Job id.
+     */
+    public void jobFinished(HadoopJobId jobId) {
+        HadoopShuffleJob 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(HadoopJobId jobId) {
+        HadoopShuffleJob 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/6423cf02/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..49cbd65
--- /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 HadoopJobId jobId;
+
+    /**
+     *
+     */
+    public HadoopShuffleAck() {
+        // No-op.
+    }
+
+    /**
+     * @param msgId Message ID.
+     */
+    public HadoopShuffleAck(long msgId, HadoopJobId jobId) {
+        assert jobId != null;
+
+        this.msgId = msgId;
+        this.jobId = jobId;
+    }
+
+    /**
+     * @return Message ID.
+     */
+    public long id() {
+        return msgId;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public HadoopJobId 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 HadoopJobId();
+
+        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/6423cf02/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..7ae52df
--- /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.HadoopJobProperty.*;
+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 HadoopJob job;
+
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /** */
+    private final boolean needPartitioner;
+
+    /** Collection of task contexts for each reduce task. */
+    private final Map<Integer, HadoopTaskContext> 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, HadoopJob 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) {
+                HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.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;
+
+        HadoopTaskContext 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 HadoopTaskOutput output(HadoopTaskContext 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 HadoopTaskInput input(HadoopTaskContext 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 HadoopTaskInput() { // 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 HadoopTaskOutput {
+        /** */
+        private final HadoopTaskOutput[] adders = new HadoopTaskOutput[maps.length()];
+
+        /** */
+        private HadoopPartitioner partitioner;
+
+        /** */
+        private final HadoopTaskContext taskCtx;
+
+        /**
+         * Constructor.
+         * @param taskCtx Task context.
+         */
+        private PartitionedOutput(HadoopTaskContext 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);
+            }
+
+            HadoopTaskOutput 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 (HadoopTaskOutput adder : adders) {
+                if (adder != null)
+                    adder.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..c350552a
--- /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 HadoopJobId jobId;
+
+    /** */
+    @GridToStringInclude
+    private int reducer;
+
+    /** */
+    private byte[] buf;
+
+    /** */
+    @GridToStringInclude
+    private int off;
+
+    /**
+     *
+     */
+    public HadoopShuffleMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param size Size.
+     */
+    public HadoopShuffleMessage(HadoopJobId 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 HadoopJobId 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 HadoopJobId();
+
+        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/6423cf02/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/6423cf02/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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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();
-        }
-    }
-}


[16/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..496a710
--- /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 HadoopInputSplit {
+    /** */
+    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/6423cf02/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/6423cf02/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..bb9cb68
--- /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 HadoopSerialization {
+    /** 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/6423cf02/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/6423cf02/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..bc7ded3
--- /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 HadoopInputSplit {
+    /** */
+    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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..534033b
--- /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(HadoopTaskInfo 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/6423cf02/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..3f8e2b6
--- /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 HadoopTaskContext input. */
+    private RecordReader reader;
+
+    /** Output writer to overriding of HadoopTaskContext output. */
+    private RecordWriter writer;
+
+    /** Output is provided by executor environment. */
+    private final HadoopTaskOutput output;
+
+    /** Input is provided by executor environment. */
+    private final HadoopTaskInput 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 HadoopTaskContext 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) {
+            HadoopInputSplit split = ctx.taskInfo().inputSplit();
+
+            if (split == null)
+                return null;
+
+            if (split instanceof HadoopFileBlock) {
+                HadoopFileBlock fileBlock = (HadoopFileBlock)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/6423cf02/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/6423cf02/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..f2f0cab
--- /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 HadoopJob {
+    /** */
+    private final JobConf jobConf;
+
+    /** */
+    private final JobContextImpl jobCtx;
+
+    /** Hadoop job ID. */
+    private final HadoopJobId jobId;
+
+    /** Job info. */
+    protected HadoopJobInfo jobInfo;
+
+    /** */
+    private final JobID hadoopJobID;
+
+    /** */
+    private final HadoopV2JobResourceManager rsrcMgr;
+
+    /** */
+    private final ConcurrentMap<T2<HadoopTaskType, Integer>, GridFutureAdapter<HadoopTaskContext>> 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(HadoopJobId 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 HadoopJobId id() {
+        return jobId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopJobInfo info() {
+        return jobInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<HadoopInputSplit> 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<HadoopInputSplit> 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);
+
+                        HadoopFileBlock 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 HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException {
+        T2<HadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
+
+        GridFutureAdapter<HadoopTaskContext> fut = ctxs.get(locTaskId);
+
+        if (fut != null)
+            return fut.get();
+
+        GridFutureAdapter<HadoopTaskContext> 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(HadoopTaskInfo.class, HadoopJob.class,
+                HadoopJobId.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();
+                    }
+                }
+
+            HadoopTaskContext res = (HadoopTaskContext)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(HadoopTaskInfo info) throws IgniteCheckedException {
+        rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cleanupTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException {
+        HadoopTaskContext 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/6423cf02/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..6f6bfa1
--- /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 HadoopJobId 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(HadoopJobId 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/6423cf02/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..2bf4292
--- /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(HadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions", "unchecked"})
+    @Override public void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException {
+        HadoopInputSplit split = info().inputSplit();
+
+        InputSplit nativeSplit;
+
+        if (split instanceof HadoopFileBlock) {
+            HadoopFileBlock block = (HadoopFileBlock)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);
+
+            HadoopJobInfo 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/6423cf02/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..36382d4
--- /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 HadoopPartitioner {
+    /** 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/6423cf02/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..250c41b
--- /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(HadoopTaskInfo 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/6423cf02/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..81587c1
--- /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(HadoopTaskInfo 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/6423cf02/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..76a3329
--- /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<HadoopInputSplit> splitJob(JobContext ctx) throws IgniteCheckedException {
+        try {
+            InputFormat<?, ?> format = ReflectionUtils.newInstance(ctx.getInputFormatClass(), ctx.getConfiguration());
+
+            assert format != null;
+
+            List<InputSplit> splits = format.getSplits(ctx);
+
+            Collection<HadoopInputSplit> res = new ArrayList<>(splits.size());
+
+            int id = 0;
+
+            for (InputSplit nativeSplit : splits) {
+                if (nativeSplit instanceof FileSplit) {
+                    FileSplit s = (FileSplit)nativeSplit;
+
+                    res.add(new HadoopFileBlock(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 HadoopFileBlock 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 HadoopFileBlock(hosts, split.getPath().toUri(), split.getStart(), split.getLength());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..5ade3fb
--- /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 HadoopTask {
+    /** Hadoop context. */
+    private HadoopV2Context hadoopCtx;
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     */
+    protected HadoopV2Task(HadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run(HadoopTaskContext 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();
+    }
+}


[57/58] [abbrv] incubator-ignite git commit: futures: api cleanup

Posted by yz...@apache.org.
futures: api cleanup


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

Branch: refs/heads/ignite-313
Commit: 10bb0da5bda6f5ef6dcd0a0f622e9c2a8af3d023
Parents: 6e52162
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Mar 5 12:00:04 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Mar 5 12:00:04 2015 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java      | 17 +++++++++--------
 .../processors/cache/GridCacheMultiTxFuture.java   | 11 +++++------
 .../internal/util/future/GridFutureAdapter.java    |  4 ++--
 3 files changed, 16 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/10bb0da5/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 968e93a..67ae808 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -560,8 +560,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     ) {
         workersCnt.increment();
 
-        Runnable c = new GridWorker(ctx.gridName(), "msg-worker", log) {
-            @Override protected void body() {
+        Runnable c = new Runnable() {
+            @Override public void run() {
                 try {
                     threadProcessingMessage(true);
 
@@ -612,8 +612,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     ) {
         workersCnt.increment();
 
-        Runnable c = new GridWorker(ctx.gridName(), "msg-worker", log) {
-            @Override protected void body() {
+        Runnable c = new Runnable() {
+            @Override public void run() {
                 try {
                     threadProcessingMessage(true);
 
@@ -811,8 +811,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         final GridCommunicationMessageSet msgSet0 = set;
 
-        Runnable c = new GridWorker(ctx.gridName(), "msg-worker", log) {
-            @Override protected void body() {
+        Runnable c = new Runnable() {
+            @Override public void run() {
                 try {
                     threadProcessingMessage(true);
 
@@ -1329,8 +1329,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                     workersCnt.increment();
 
-                    pool(msgSet.policy()).execute(new GridWorker(ctx.gridName(), "msg-worker", log) {
-                        @Override protected void body() {
+                    pool(msgSet.policy()).execute(
+                        new Runnable() {
+                            @Override public void run() {
                             try {
                                 unwindMessageSet(msgSet, lsnrs0);
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/10bb0da5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
index 71ba123..7df184e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
@@ -35,19 +35,18 @@ public final class GridCacheMultiTxFuture<K, V> extends GridFutureAdapter<Boolea
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
+    /** Logger. */
+    private static IgniteLogger log;
+
     /** */
     private Set<IgniteInternalTx<K, V>> remainingTxs;
 
-    /** Logger. */
-    private IgniteLogger log;
-
     /**
      * @param cctx Cache context.
      */
     public GridCacheMultiTxFuture(GridCacheContext<K, V> cctx) {
-        super();
-
-        log = U.logger(cctx.kernalContext(),  logRef, GridCacheMultiTxFuture.class);
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridCacheMultiTxFuture.class);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/10bb0da5/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index b35d18a..21ad1ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -204,9 +204,9 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
             lsnr = null;
         }
 
-        assert lsnr == null;
+        assert lsnrs0 == null;
 
-        notifyListener(lsnr);
+        notifyListener(lsnrs0);
     }
 
     /**


[45/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
IGNITE-386: Squashed changes.


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

Branch: refs/heads/ignite-313
Commit: 6423cf027e9639b5260bd0cfd6a5e8eb82cd7acd
Parents: 162760b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Mar 5 10:15:16 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 5 10:15:22 2015 +0300

----------------------------------------------------------------------
 bin/setup-hadoop.bat                            |    2 +-
 bin/setup-hadoop.sh                             |    2 +-
 config/hadoop/default-config.xml                |   10 +-
 docs/core-site.ignite.xml                       |    6 +-
 examples/config/filesystem/core-site.xml        |    4 +-
 examples/config/filesystem/example-igfs.xml     |    4 +-
 .../ignite/examples/igfs/IgfsExample.java       |   18 +-
 .../examples/igfs/IgfsMapReduceExample.java     |    6 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   13 +-
 .../org/apache/ignite/IgniteFileSystem.java     |  457 ++++
 .../main/java/org/apache/ignite/IgniteFs.java   |  346 ---
 .../configuration/FileSystemConfiguration.java  |  805 ++++++
 .../configuration/HadoopConfiguration.java      |  173 ++
 .../ignite/configuration/IgfsConfiguration.java |  807 ------
 .../configuration/IgniteConfiguration.java      |   19 +-
 .../main/java/org/apache/ignite/igfs/Igfs.java  |  216 --
 .../apache/ignite/igfs/IgfsBlockLocation.java   |    2 +-
 .../java/org/apache/ignite/igfs/IgfsFile.java   |    2 +-
 .../org/apache/ignite/igfs/IgfsInputStream.java |    6 +-
 .../org/apache/ignite/igfs/IgfsMetrics.java     |    2 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    8 +-
 .../java/org/apache/ignite/igfs/IgfsReader.java |   38 -
 .../mapreduce/IgfsInputStreamJobAdapter.java    |    4 +-
 .../apache/ignite/igfs/mapreduce/IgfsJob.java   |    4 +-
 .../igfs/mapreduce/IgfsRecordResolver.java      |    2 +-
 .../apache/ignite/igfs/mapreduce/IgfsTask.java  |    4 +-
 .../ignite/igfs/mapreduce/IgfsTaskArgs.java     |    2 +-
 .../IgfsByteDelimiterRecordResolver.java        |    3 +-
 .../records/IgfsFixedLengthRecordResolver.java  |    2 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  201 ++
 ...fsSecondaryFileSystemPositionedReadable.java |   38 +
 .../apache/ignite/igfs/secondary/package.html   |   24 +
 .../ignite/internal/GridKernalContext.java      |    2 +-
 .../ignite/internal/GridKernalContextImpl.java  |    8 +-
 .../ignite/internal/IgniteComponentType.java    |    4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |    4 +-
 .../apache/ignite/internal/IgniteKernal.java    |   14 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    8 +-
 .../processors/cache/GridCacheAdapter.java      |    4 +-
 .../processors/cache/GridCacheProcessor.java    |    4 +-
 .../processors/cache/GridCacheUtils.java        |    4 +-
 .../internal/processors/hadoop/GridHadoop.java  |   86 -
 .../hadoop/GridHadoopConfiguration.java         |  172 --
 .../processors/hadoop/GridHadoopCounter.java    |   44 -
 .../hadoop/GridHadoopCounterWriter.java         |   36 -
 .../processors/hadoop/GridHadoopCounters.java   |   49 -
 .../processors/hadoop/GridHadoopFileBlock.java  |  162 --
 .../processors/hadoop/GridHadoopInputSplit.java |   54 -
 .../processors/hadoop/GridHadoopJob.java        |  102 -
 .../processors/hadoop/GridHadoopJobId.java      |  103 -
 .../processors/hadoop/GridHadoopJobInfo.java    |   83 -
 .../processors/hadoop/GridHadoopJobPhase.java   |   38 -
 .../hadoop/GridHadoopJobProperty.java           |  138 -
 .../processors/hadoop/GridHadoopJobStatus.java  |  207 --
 .../hadoop/GridHadoopMapReducePlan.java         |   80 -
 .../hadoop/GridHadoopMapReducePlanner.java      |   40 -
 .../hadoop/GridHadoopPartitioner.java           |   33 -
 .../hadoop/GridHadoopSerialization.java         |   54 -
 .../processors/hadoop/GridHadoopTask.java       |   72 -
 .../hadoop/GridHadoopTaskContext.java           |  189 --
 .../processors/hadoop/GridHadoopTaskInfo.java   |  153 --
 .../processors/hadoop/GridHadoopTaskInput.java  |   55 -
 .../processors/hadoop/GridHadoopTaskOutput.java |   40 -
 .../processors/hadoop/GridHadoopTaskType.java   |   56 -
 .../internal/processors/hadoop/Hadoop.java      |   88 +
 .../processors/hadoop/HadoopFileBlock.java      |  162 ++
 .../processors/hadoop/HadoopInputSplit.java     |   54 +
 .../internal/processors/hadoop/HadoopJob.java   |  102 +
 .../internal/processors/hadoop/HadoopJobId.java |  103 +
 .../processors/hadoop/HadoopJobInfo.java        |   83 +
 .../processors/hadoop/HadoopJobPhase.java       |   38 +
 .../processors/hadoop/HadoopJobProperty.java    |  138 +
 .../processors/hadoop/HadoopJobStatus.java      |  207 ++
 .../processors/hadoop/HadoopMapReducePlan.java  |   80 +
 .../hadoop/HadoopMapReducePlanner.java          |   40 +
 .../processors/hadoop/HadoopNoopProcessor.java  |   76 +
 .../processors/hadoop/HadoopPartitioner.java    |   33 +
 .../hadoop/HadoopProcessorAdapter.java          |   96 +
 .../processors/hadoop/HadoopSerialization.java  |   54 +
 .../internal/processors/hadoop/HadoopTask.java  |   72 +
 .../processors/hadoop/HadoopTaskContext.java    |  190 ++
 .../processors/hadoop/HadoopTaskInfo.java       |  153 ++
 .../processors/hadoop/HadoopTaskInput.java      |   55 +
 .../processors/hadoop/HadoopTaskOutput.java     |   40 +
 .../processors/hadoop/HadoopTaskType.java       |   56 +
 .../hadoop/IgniteHadoopNoopProcessor.java       |   74 -
 .../hadoop/IgniteHadoopProcessorAdapter.java    |   94 -
 .../hadoop/counter/HadoopCounter.java           |   44 +
 .../hadoop/counter/HadoopCounterWriter.java     |   37 +
 .../hadoop/counter/HadoopCounters.java          |   49 +
 .../internal/processors/igfs/IgfsAsyncImpl.java |    9 +-
 .../internal/processors/igfs/IgfsContext.java   |    6 +-
 .../processors/igfs/IgfsDataManager.java        |    5 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   28 +-
 .../internal/processors/igfs/IgfsFileInfo.java  |    4 +-
 .../internal/processors/igfs/IgfsImpl.java      |   42 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |    4 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |    5 +-
 .../internal/processors/igfs/IgfsJobImpl.java   |    2 +-
 .../processors/igfs/IgfsMetaManager.java        |   27 +-
 .../processors/igfs/IgfsNoopProcessor.java      |    4 +-
 .../processors/igfs/IgfsOutputStreamImpl.java   |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |   26 +-
 .../processors/igfs/IgfsProcessorAdapter.java   |    4 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |  121 +
 .../IgfsSecondaryInputStreamDescriptor.java     |    8 +-
 .../processors/igfs/IgfsServerManager.java      |    4 +-
 .../ignite/internal/visor/igfs/VisorIgfs.java   |    4 +-
 .../visor/igfs/VisorIgfsProfilerClearTask.java  |    2 +-
 .../visor/node/VisorGridConfiguration.java      |    2 +-
 .../visor/node/VisorIgfsConfiguration.java      |    9 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    2 +-
 .../internal/visor/util/VisorTaskUtils.java     |    4 +-
 .../hadoop/core-site-loopback-secondary.xml     |    4 +-
 .../test/config/hadoop/core-site-loopback.xml   |    4 +-
 .../test/config/hadoop/core-site-secondary.xml  |    4 +-
 .../core/src/test/config/hadoop/core-site.xml   |    4 +-
 modules/core/src/test/config/igfs-loopback.xml  |    4 +-
 modules/core/src/test/config/igfs-shmem.xml     |    4 +-
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   10 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |    4 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |    8 +-
 .../igfs/IgfsFragmentizerTopologySelfTest.java  |    2 +-
 ...heIgfsPerBlockLruEvictionPolicySelfTest.java |  485 ----
 .../processors/igfs/IgfsAbstractSelfTest.java   |   85 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |  485 ++++
 .../processors/igfs/IgfsCacheSelfTest.java      |    4 +-
 .../igfs/IgfsDataManagerSelfTest.java           |    5 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   12 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |    4 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   22 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   14 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   10 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |   50 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |   10 +-
 ...dpointRegistrationOnLinuxAndMacSelfTest.java |    4 +-
 ...pcEndpointRegistrationOnWindowsSelfTest.java |    2 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |    4 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   14 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   10 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |    6 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |    4 +-
 .../ignite/testframework/junits/IgniteMock.java |    4 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    2 +-
 .../client/hadoop/GridHadoopClientProtocol.java |  334 ---
 .../GridHadoopClientProtocolProvider.java       |  137 -
 .../counter/GridHadoopClientCounterGroup.java   |  121 -
 .../counter/GridHadoopClientCounters.java       |  217 --
 .../apache/ignite/client/hadoop/package.html    |   24 -
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   93 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  414 +++
 .../org/apache/ignite/hadoop/fs/package.html    |   24 +
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 1237 +++++++++
 .../org/apache/ignite/hadoop/fs/v1/package.html |   24 +
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  982 ++++++++
 .../org/apache/ignite/hadoop/fs/v2/package.html |   24 +
 .../IgniteHadoopClientProtocolProvider.java     |  138 +
 .../mapreduce/IgniteHadoopMapReducePlanner.java |  435 ++++
 .../apache/ignite/hadoop/mapreduce/package.html |   24 +
 .../java/org/apache/ignite/hadoop/package.html  |   24 +
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  414 ---
 .../igfs/hadoop/IgfsHadoopParameters.java       |   94 -
 .../org/apache/ignite/igfs/hadoop/package.html  |   24 -
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    | 1234 ---------
 .../apache/ignite/igfs/hadoop/v1/package.html   |   24 -
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |  982 --------
 .../apache/ignite/igfs/hadoop/v2/package.html   |   24 -
 .../java/org/apache/ignite/igfs/package.html    |   24 -
 .../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 ----
 .../ignite/internal/igfs/hadoop/package.html    |   24 -
 .../apache/ignite/internal/igfs/package.html    |   24 -
 .../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        |  197 ++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  163 ++
 .../internal/processors/hadoop/HadoopImpl.java  |  134 +
 .../hadoop/HadoopMapReduceCounterGroup.java     |  121 +
 .../hadoop/HadoopMapReduceCounters.java         |  216 ++
 .../processors/hadoop/HadoopProcessor.java      |  227 ++
 .../internal/processors/hadoop/HadoopSetup.java |  505 ++++
 .../hadoop/HadoopTaskCancelledException.java    |   35 +
 .../internal/processors/hadoop/HadoopUtils.java |  308 +++
 .../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    |  127 +
 .../hadoop/counter/HadoopCountersImpl.java      |  197 ++
 .../hadoop/counter/HadoopLongCounter.java       |   90 +
 .../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 +
 .../processors/hadoop/fs/HadoopParameters.java  |   94 +
 .../hadoop/fs/HadoopRawLocalFileSystem.java     |  304 +++
 .../processors/hadoop/igfs/HadoopIgfs.java      |  198 ++
 .../igfs/HadoopIgfsCommunicationException.java  |   57 +
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |  210 ++
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |   88 +
 .../hadoop/igfs/HadoopIgfsFuture.java           |   94 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  409 +++
 .../hadoop/igfs/HadoopIgfsInputStream.java      |  626 +++++
 .../processors/hadoop/igfs/HadoopIgfsIo.java    |   76 +
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |  599 +++++
 .../hadoop/igfs/HadoopIgfsIpcIoListener.java    |   36 +
 .../hadoop/igfs/HadoopIgfsJclLogger.java        |  115 +
 .../hadoop/igfs/HadoopIgfsOutProc.java          |  466 ++++
 .../hadoop/igfs/HadoopIgfsOutputStream.java     |  201 ++
 .../hadoop/igfs/HadoopIgfsProperties.java       |   88 +
 .../hadoop/igfs/HadoopIgfsProxyInputStream.java |  335 +++
 .../igfs/HadoopIgfsProxyOutputStream.java       |  165 ++
 ...fsSecondaryFileSystemPositionedReadable.java |  104 +
 .../hadoop/igfs/HadoopIgfsStreamDelegate.java   |   96 +
 .../igfs/HadoopIgfsStreamEventListener.java     |   39 +
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |  131 +
 .../hadoop/igfs/HadoopIgfsWrapper.java          |  511 ++++
 .../jobtracker/GridHadoopJobMetadata.java       |  305 ---
 .../hadoop/jobtracker/GridHadoopJobTracker.java | 1625 ------------
 .../hadoop/jobtracker/HadoopJobMetadata.java    |  306 +++
 .../hadoop/jobtracker/HadoopJobTracker.java     | 1626 ++++++++++++
 .../hadoop/message/GridHadoopMessage.java       |   27 -
 .../hadoop/message/HadoopMessage.java           |   27 +
 .../planner/GridHadoopDefaultMapReducePlan.java |  107 -
 .../GridHadoopDefaultMapReducePlanner.java      |  434 ----
 .../planner/HadoopDefaultMapReducePlan.java     |  107 +
 .../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      |  333 +++
 .../proto/HadoopProtocolJobCountersTask.java    |   46 +
 .../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/GridHadoopShuffle.java       |  256 --
 .../hadoop/shuffle/GridHadoopShuffleAck.java    |   91 -
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |  593 -----
 .../shuffle/GridHadoopShuffleMessage.java       |  242 --
 .../hadoop/shuffle/HadoopShuffle.java           |  256 ++
 .../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 +
 .../GridHadoopEmbeddedTaskExecutor.java         |  146 --
 .../taskexecutor/GridHadoopExecutorService.java |  232 --
 .../taskexecutor/GridHadoopRunnableTask.java    |  268 --
 .../GridHadoopTaskExecutorAdapter.java          |   57 -
 .../taskexecutor/GridHadoopTaskState.java       |   38 -
 .../taskexecutor/GridHadoopTaskStatus.java      |  114 -
 .../HadoopEmbeddedTaskExecutor.java             |  146 ++
 .../taskexecutor/HadoopExecutorService.java     |  231 ++
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  268 ++
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   57 +
 .../hadoop/taskexecutor/HadoopTaskState.java    |   38 +
 .../hadoop/taskexecutor/HadoopTaskStatus.java   |  114 +
 .../GridHadoopExternalTaskExecutor.java         |  960 -------
 .../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    |  960 +++++++
 .../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     |  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          |  444 ++++
 .../hadoop/v2/HadoopWritableSerialization.java  |   74 +
 ...op.mapreduce.protocol.ClientProtocolProvider |    2 +-
 ...ridHadoopClientProtocolEmbeddedSelfTest.java |   34 -
 .../GridHadoopClientProtocolSelfTest.java       |  633 -----
 .../HadoopClientProtocolEmbeddedSelfTest.java   |   34 +
 .../hadoop/HadoopClientProtocolSelfTest.java    |  635 +++++
 .../HadoopIgfs20FileSystemAbstractSelfTest.java | 1967 +++++++++++++++
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   74 +
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   74 +
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |  305 +++
 .../igfs/HadoopIgfsDualAsyncSelfTest.java       |   32 +
 .../ignite/igfs/HadoopIgfsDualSyncSelfTest.java |   32 +
 ...oopSecondaryFileSystemConfigurationTest.java |   27 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   54 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java | 1967 ---------------
 ...doop20FileSystemLoopbackPrimarySelfTest.java |   74 -
 ...sHadoop20FileSystemShmemPrimarySelfTest.java |   74 -
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |  304 ---
 .../igfs/IgfsHadoopDualAsyncSelfTest.java       |   32 -
 .../ignite/igfs/IgfsHadoopDualSyncSelfTest.java |   32 -
 .../IgfsHadoopFileSystemAbstractSelfTest.java   | 2366 ------------------
 .../IgfsHadoopFileSystemClientSelfTest.java     |  199 --
 .../IgfsHadoopFileSystemHandshakeSelfTest.java  |  311 ---
 .../IgfsHadoopFileSystemIpcCacheSelfTest.java   |  207 --
 .../IgfsHadoopFileSystemLoggerSelfTest.java     |  287 ---
 ...IgfsHadoopFileSystemLoggerStateSelfTest.java |  325 ---
 ...adoopFileSystemLoopbackAbstractSelfTest.java |   46 -
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |   33 -
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |   33 -
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |   33 -
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |   34 -
 ...SystemLoopbackExternalDualAsyncSelfTest.java |   33 -
 ...eSystemLoopbackExternalDualSyncSelfTest.java |   33 -
 ...leSystemLoopbackExternalPrimarySelfTest.java |   33 -
 ...SystemLoopbackExternalSecondarySelfTest.java |   34 -
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |  319 ---
 ...fsHadoopFileSystemShmemAbstractSelfTest.java |   88 -
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 -
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 -
 ...ileSystemShmemExternalDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemExternalDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemExternalPrimarySelfTest.java |   33 -
 ...ileSystemShmemExternalSecondarySelfTest.java |   33 -
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java | 2366 ++++++++++++++++++
 .../IgniteHadoopFileSystemClientSelfTest.java   |  199 ++
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  311 +++
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |  207 ++
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |  287 +++
 ...niteHadoopFileSystemLoggerStateSelfTest.java |  325 +++
 ...adoopFileSystemLoopbackAbstractSelfTest.java |   46 +
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |   33 +
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |   33 +
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |   33 +
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |   34 +
 ...SystemLoopbackExternalDualAsyncSelfTest.java |   33 +
 ...eSystemLoopbackExternalDualSyncSelfTest.java |   33 +
 ...leSystemLoopbackExternalPrimarySelfTest.java |   33 +
 ...SystemLoopbackExternalSecondarySelfTest.java |   34 +
 ...teHadoopFileSystemSecondaryModeSelfTest.java |  319 +++
 ...teHadoopFileSystemShmemAbstractSelfTest.java |   88 +
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 +
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 +
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 +
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 +
 ...ileSystemShmemExternalDualAsyncSelfTest.java |   33 +
 ...FileSystemShmemExternalDualSyncSelfTest.java |   33 +
 ...pFileSystemShmemExternalPrimarySelfTest.java |   33 +
 ...ileSystemShmemExternalSecondarySelfTest.java |   33 +
 .../hadoop/GridHadoopAbstractSelfTest.java      |  222 --
 .../hadoop/GridHadoopAbstractWordCountTest.java |  138 -
 .../hadoop/GridHadoopClassLoaderTest.java       |   69 -
 .../hadoop/GridHadoopCommandLineTest.java       |  440 ----
 ...idHadoopDefaultMapReducePlannerSelfTest.java | 1005 --------
 .../hadoop/GridHadoopFileSystemsTest.java       |  177 --
 .../hadoop/GridHadoopGroupingTest.java          |  286 ---
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |  330 ---
 .../GridHadoopMapReduceEmbeddedSelfTest.java    |  245 --
 .../hadoop/GridHadoopMapReduceTest.java         |  195 --
 .../hadoop/GridHadoopPopularWordsTest.java      |  294 ---
 .../GridHadoopSerializationWrapperSelfTest.java |   74 -
 .../processors/hadoop/GridHadoopSharedMap.java  |   67 -
 .../hadoop/GridHadoopSortingExternalTest.java   |   32 -
 .../hadoop/GridHadoopSortingTest.java           |  281 ---
 .../hadoop/GridHadoopSplitWrapperSelfTest.java  |   68 -
 .../processors/hadoop/GridHadoopStartup.java    |   55 -
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |  551 ----
 .../hadoop/GridHadoopTasksAllVersionsTest.java  |  259 --
 .../hadoop/GridHadoopTasksV1Test.java           |   57 -
 .../hadoop/GridHadoopTasksV2Test.java           |   75 -
 .../GridHadoopTestRoundRobinMrPlanner.java      |   66 -
 .../hadoop/GridHadoopTestTaskContext.java       |  219 --
 .../processors/hadoop/GridHadoopTestUtils.java  |  102 -
 .../hadoop/GridHadoopV2JobSelfTest.java         |   88 -
 .../hadoop/GridHadoopValidationSelfTest.java    |   53 -
 .../hadoop/HadoopAbstractSelfTest.java          |  222 ++
 .../hadoop/HadoopAbstractWordCountTest.java     |  138 +
 .../hadoop/HadoopClassLoaderTest.java           |   69 +
 .../hadoop/HadoopCommandLineTest.java           |  440 ++++
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 1006 ++++++++
 .../hadoop/HadoopFileSystemsTest.java           |  177 ++
 .../processors/hadoop/HadoopGroupingTest.java   |  287 +++
 .../hadoop/HadoopJobTrackerSelfTest.java        |  331 +++
 .../hadoop/HadoopMapReduceEmbeddedSelfTest.java |  246 ++
 .../processors/hadoop/HadoopMapReduceTest.java  |  197 ++
 .../hadoop/HadoopPopularWordsTest.java          |  294 +++
 .../HadoopSerializationWrapperSelfTest.java     |   74 +
 .../processors/hadoop/HadoopSharedMap.java      |   67 +
 .../hadoop/HadoopSortingExternalTest.java       |   34 +
 .../processors/hadoop/HadoopSortingTest.java    |  282 +++
 .../hadoop/HadoopSplitWrapperSelfTest.java      |   68 +
 .../processors/hadoop/HadoopStartup.java        |   54 +
 .../hadoop/HadoopTaskExecutionSelfTest.java     |  551 ++++
 .../hadoop/HadoopTasksAllVersionsTest.java      |  259 ++
 .../processors/hadoop/HadoopTasksV1Test.java    |   57 +
 .../processors/hadoop/HadoopTasksV2Test.java    |   75 +
 .../hadoop/HadoopTestRoundRobinMrPlanner.java   |   66 +
 .../hadoop/HadoopTestTaskContext.java           |  219 ++
 .../processors/hadoop/HadoopTestUtils.java      |  102 +
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   88 +
 .../hadoop/HadoopValidationSelfTest.java        |   53 +
 .../hadoop/examples/GridHadoopWordCount1.java   |   88 -
 .../examples/GridHadoopWordCount1Map.java       |   62 -
 .../examples/GridHadoopWordCount1Reduce.java    |   51 -
 .../hadoop/examples/GridHadoopWordCount2.java   |   95 -
 .../examples/GridHadoopWordCount2Mapper.java    |   72 -
 .../examples/GridHadoopWordCount2Reducer.java   |   70 -
 .../hadoop/examples/HadoopWordCount1.java       |   88 +
 .../hadoop/examples/HadoopWordCount1Map.java    |   62 +
 .../hadoop/examples/HadoopWordCount1Reduce.java |   51 +
 .../hadoop/examples/HadoopWordCount2.java       |   95 +
 .../hadoop/examples/HadoopWordCount2Mapper.java |   72 +
 .../examples/HadoopWordCount2Reducer.java       |   70 +
 .../collections/GridHadoopAbstractMapTest.java  |  152 --
 ...ridHadoopConcurrentHashMultimapSelftest.java |  267 --
 .../collections/GridHadoopHashMapSelfTest.java  |  170 --
 .../collections/GridHadoopSkipListSelfTest.java |  303 ---
 .../collections/HadoopAbstractMapTest.java      |  154 ++
 .../HadoopConcurrentHashMultimapSelftest.java   |  267 ++
 .../collections/HadoopHashMapSelfTest.java      |  170 ++
 .../collections/HadoopSkipListSelfTest.java     |  303 +++
 .../streams/GridHadoopDataStreamSelfTest.java   |  151 --
 .../streams/HadoopDataStreamSelfTest.java       |  151 ++
 .../GridHadoopExecutorServiceTest.java          |  119 -
 .../taskexecutor/HadoopExecutorServiceTest.java |  119 +
 ...GridHadoopExternalTaskExecutionSelfTest.java |  210 --
 .../HadoopExternalTaskExecutionSelfTest.java    |  211 ++
 ...GridHadoopExternalCommunicationSelfTest.java |  209 --
 .../HadoopExternalCommunicationSelfTest.java    |  209 ++
 .../testsuites/IgniteHadoopTestSuite.java       |   82 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   22 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    4 +-
 pom.xml                                         |   12 +-
 561 files changed, 47565 insertions(+), 47291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/config/hadoop/default-config.xml
----------------------------------------------------------------------
diff --git a/config/hadoop/default-config.xml b/config/hadoop/default-config.xml
index a264749..65a281e 100644
--- a/config/hadoop/default-config.xml
+++ b/config/hadoop/default-config.xml
@@ -52,7 +52,7 @@
     <!--
         Abstract IGFS file system configuration to be used as a template.
     -->
-    <bean id="igfsCfgBase" class="org.apache.ignite.configuration.IgfsConfiguration" abstract="true">
+    <bean id="igfsCfgBase" class="org.apache.ignite.configuration.FileSystemConfiguration" abstract="true">
         <!-- Must correlate with cache affinity mapper. -->
         <property name="blockSize" value="#{128 * 1024}"/>
         <property name="perNodeBatchSize" value="512"/>
@@ -94,7 +94,7 @@
             Apache Hadoop Accelerator configuration.
         -->
         <property name="hadoopConfiguration">
-            <bean class="org.apache.ignite.internal.processors.hadoop.GridHadoopConfiguration">
+            <bean class="org.apache.ignite.configuration.HadoopConfiguration">
                 <!-- Information about finished jobs will be kept for 30 seconds. -->
                 <property name="finishedJobInfoTtl" value="30000"/>
             </bean>
@@ -112,9 +112,9 @@
         <!--
             Configure one IGFS file system instance named "igfs" on this node.
         -->
-        <property name="igfsConfiguration">
+        <property name="fileSystemConfiguration">
             <list>
-                <bean class="org.apache.ignite.configuration.IgfsConfiguration" parent="igfsCfgBase">
+                <bean class="org.apache.ignite.configuration.FileSystemConfiguration" parent="igfsCfgBase">
                     <property name="name" value="igfs"/>
 
                     <!-- Caches with these names must be configured. -->
@@ -135,7 +135,7 @@
                     <property name="defaultMode" value="PROXY"/>
 
                     <property name="secondaryFileSystem">
-                        <bean class="org.apache.ignite.igfs.hadoop.IgfsHadoopFileSystemWrapper">
+                        <bean class="org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem">
                             <constructor-arg name="uri"     value="hdfs://1.2.3.4:9000"/>
                             <constructor-arg name="cfgPath" value="/opt/hadoop-server/etc/hadoop/core-site.xml"/>
                         </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/docs/core-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/core-site.ignite.xml b/docs/core-site.ignite.xml
index ed11a79..8b8e634 100644
--- a/docs/core-site.ignite.xml
+++ b/docs/core-site.ignite.xml
@@ -48,7 +48,7 @@
     -->
     <property>
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <!--
@@ -56,7 +56,7 @@
     -->
     <property>
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 
     <!--
@@ -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/6423cf02/examples/config/filesystem/core-site.xml
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/core-site.xml b/examples/config/filesystem/core-site.xml
index 7c6cfaa..b6f0291 100644
--- a/examples/config/filesystem/core-site.xml
+++ b/examples/config/filesystem/core-site.xml
@@ -31,12 +31,12 @@
     <property>
         <!-- FS driver class for the 'igfs://' URIs. -->
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <property>
         <!-- FS driver class for the 'igfs://' URIs in Hadoop2.x -->
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/examples/config/filesystem/example-igfs.xml
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/example-igfs.xml b/examples/config/filesystem/example-igfs.xml
index 30cf51e..d8ccd34 100644
--- a/examples/config/filesystem/example-igfs.xml
+++ b/examples/config/filesystem/example-igfs.xml
@@ -63,9 +63,9 @@
             </bean>
         </property>
 
-        <property name="igfsConfiguration">
+        <property name="fileSystemConfiguration">
             <list>
-                <bean class="org.apache.ignite.configuration.IgfsConfiguration">
+                <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
                     <property name="metaCacheName" value="igfs-meta"/>
                     <property name="dataCacheName" value="igfs-data"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsExample.java b/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsExample.java
index 56f3fa4..8bfd38d 100644
--- a/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsExample.java
@@ -25,7 +25,7 @@ import java.io.*;
 import java.util.*;
 
 /**
- * Example that shows usage of {@link org.apache.ignite.IgniteFs} API. It starts a node with {@code IgniteFs}
+ * Example that shows usage of {@link org.apache.ignite.IgniteFileSystem} API. It starts a node with {@code IgniteFs}
  * configured and performs several file system operations (create, write, append, read and delete
  * files, create, list and delete directories).
  * <p>
@@ -50,7 +50,7 @@ public final class IgfsExample {
 
         try {
             // Get an instance of Ignite File System.
-            IgniteFs fs = ignite.fileSystem("igfs");
+            IgniteFileSystem fs = ignite.fileSystem("igfs");
 
             // Working directory path.
             IgfsPath workDir = new IgfsPath("/examples/fs");
@@ -107,7 +107,7 @@ public final class IgfsExample {
      * @param path File or directory path.
      * @throws IgniteException In case of error.
      */
-    private static void delete(IgniteFs fs, IgfsPath path) throws IgniteException {
+    private static void delete(IgniteFileSystem fs, IgfsPath path) throws IgniteException {
         assert fs != null;
         assert path != null;
 
@@ -139,7 +139,7 @@ public final class IgfsExample {
      * @param path Directory path.
      * @throws IgniteException In case of error.
      */
-    private static void mkdirs(IgniteFs fs, IgfsPath path) throws IgniteException {
+    private static void mkdirs(IgniteFileSystem fs, IgfsPath path) throws IgniteException {
         assert fs != null;
         assert path != null;
 
@@ -166,7 +166,7 @@ public final class IgfsExample {
      * @throws IgniteException If file can't be created.
      * @throws IOException If data can't be written.
      */
-    private static void create(IgniteFs fs, IgfsPath path, @Nullable byte[] data)
+    private static void create(IgniteFileSystem fs, IgfsPath path, @Nullable byte[] data)
         throws IgniteException, IOException {
         assert fs != null;
         assert path != null;
@@ -195,7 +195,7 @@ public final class IgfsExample {
      * @throws IgniteException If file can't be created.
      * @throws IOException If data can't be written.
      */
-    private static void append(IgniteFs fs, IgfsPath path, byte[] data) throws IgniteException, IOException {
+    private static void append(IgniteFileSystem fs, IgfsPath path, byte[] data) throws IgniteException, IOException {
         assert fs != null;
         assert path != null;
         assert data != null;
@@ -220,7 +220,7 @@ public final class IgfsExample {
      * @throws IgniteException If file can't be opened.
      * @throws IOException If data can't be read.
      */
-    private static void read(IgniteFs fs, IgfsPath path) throws IgniteException, IOException {
+    private static void read(IgniteFileSystem fs, IgfsPath path) throws IgniteException, IOException {
         assert fs != null;
         assert path != null;
         assert fs.info(path).isFile();
@@ -242,7 +242,7 @@ public final class IgfsExample {
      * @param path Directory path.
      * @throws IgniteException In case of error.
      */
-    private static void list(IgniteFs fs, IgfsPath path) throws IgniteException {
+    private static void list(IgniteFileSystem fs, IgfsPath path) throws IgniteException {
         assert fs != null;
         assert path != null;
         assert fs.info(path).isDirectory();
@@ -271,7 +271,7 @@ public final class IgfsExample {
      * @param path File or directory path.
      * @throws IgniteException In case of error.
      */
-    private static void printInfo(IgniteFs fs, IgfsPath path) throws IgniteException {
+    private static void printInfo(IgniteFileSystem fs, IgfsPath path) throws IgniteException {
         System.out.println();
         System.out.println("Information for " + path + ": " + fs.info(path));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsMapReduceExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsMapReduceExample.java b/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsMapReduceExample.java
index 22a66a3..ed0abe4 100644
--- a/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsMapReduceExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/igfs/IgfsMapReduceExample.java
@@ -61,7 +61,7 @@ public class IgfsMapReduceExample {
                 String regexStr = args[1];
 
                 // Get an instance of Ignite File System.
-                IgniteFs fs = ignite.fileSystem("igfs");
+                IgniteFileSystem fs = ignite.fileSystem("igfs");
 
                 // Working directory path.
                 IgfsPath workDir = new IgfsPath("/examples/fs");
@@ -94,7 +94,7 @@ public class IgfsMapReduceExample {
      * @param file File to write.
      * @throws Exception In case of exception.
      */
-    private static void writeFile(IgniteFs fs, IgfsPath fsPath, File file) throws Exception {
+    private static void writeFile(IgniteFileSystem fs, IgfsPath fsPath, File file) throws Exception {
         System.out.println();
         System.out.println("Copying file to IGFS: " + file);
 
@@ -173,7 +173,7 @@ public class IgfsMapReduceExample {
         }
 
         /**  {@inheritDoc} */
-        @Override public Object execute(IgniteFs igfs, IgfsRangeInputStream in) throws IgniteException, IOException {
+        @Override public Object execute(IgniteFileSystem igfs, IgfsRangeInputStream in) throws IgniteException, IOException {
             Collection<Line> res = null;
 
             long start = in.startOffset();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 8851d8f..31b827e 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -43,7 +43,7 @@ import java.util.concurrent.*;
  * <ul>
  * <li>{@link org.apache.ignite.cache.GridCache} - functionality for in-memory distributed cache.</li>
  * <li>{@link IgniteDataLoader} - functionality for loading data large amounts of data into cache.</li>
- * <li>{@link IgniteFs} - functionality for distributed Hadoop-compliant in-memory file system and map-reduce.</li>
+ * <li>{@link IgniteFileSystem} - functionality for distributed Hadoop-compliant in-memory file system and map-reduce.</li>
  * <li>{@link IgniteStreamer} - functionality for streaming events workflow with queries and indexes into rolling windows.</li>
  * <li>{@link IgniteScheduler} - functionality for scheduling jobs using UNIX Cron syntax.</li>
  * <li>{@link IgniteCompute} - functionality for executing tasks and closures on all grid nodes (inherited form {@link ClusterGroup}).</li>
@@ -213,7 +213,7 @@ public interface Ignite extends AutoCloseable {
     public <K, V> IgniteDataLoader<K, V> dataLoader(@Nullable String cacheName);
 
     /**
-     * Gets an instance of IGFS - Ignite In-Memory File System, if one is not
+     * Gets an instance of IGFS (Ignite In-Memory File System). If one is not
      * configured then {@link IllegalArgumentException} will be thrown.
      * <p>
      * IGFS is fully compliant with Hadoop {@code FileSystem} APIs and can
@@ -222,15 +222,16 @@ public interface Ignite extends AutoCloseable {
      *
      * @param name IGFS name.
      * @return IGFS instance.
+     * @throws IllegalArgumentException If IGFS with such name is not configured.
      */
-    public IgniteFs fileSystem(String name);
+    public IgniteFileSystem fileSystem(String name);
 
     /**
-     * Gets all instances of the grid file systems.
+     * Gets all instances of IGFS (Ignite In-Memory File System).
      *
-     * @return Collection of grid file systems instances.
+     * @return Collection of IGFS instances.
      */
-    public Collection<IgniteFs> fileSystems();
+    public Collection<IgniteFileSystem> fileSystems();
 
     /**
      * Gets an instance of streamer by name, if one does not exist then

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
new file mode 100644
index 0000000..d221ae2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java
@@ -0,0 +1,457 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.mapreduce.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * <b>IG</b>nite <b>F</b>ile <b>S</b>ystem API. It provides a typical file system "view" on a particular cache:
+ * <ul>
+ *     <li>list directories or get information for a single path</li>
+ *     <li>create/move/delete files or directories</li>
+ *     <li>write/read data streams into/from files</li>
+ * </ul>
+ * The data of each file is split on separate data blocks and stored in the cache.
+ * You can access file's data with a standard Java streaming API. Moreover, for each part
+ * of the file you can calculate an affinity and process file's content on corresponding
+ * nodes to escape unnecessary networking.
+ * <p/>
+ * This API is fully thread-safe and you can use it from several threads.
+ * <h1 class="header">IGFS Configuration</h1>
+ * The simplest way to run a Ignite node with configured file system is to pass
+ * special configuration file included in Ignite distribution to {@code ignite.sh} or
+ * {@code ignite.bat} scripts, like this: {@code ignite.sh config/hadoop/default-config.xml}
+ * <p>
+ * {@code IGFS} can be started as a data node or as a client node. Data node is responsible for
+ * caching data, while client node is responsible for basic file system operations and accessing
+ * data nodes remotely. When used as Hadoop file system, clients nodes usually started together
+ * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
+ * started together with Hadoop {@code task-tracker} processes.
+ * <h1 class="header">Integration With Hadoop</h1>
+ * In addition to direct file system API, {@code IGFS} can be integrated with {@code Hadoop} by
+ * plugging in as {@code Hadoop FileSystem}. Refer to
+ * {@code org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem} or
+ * {@code org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem} for more information.
+ * <p>
+ * <b>NOTE:</b> integration with Hadoop is available only in {@code In-Memory Accelerator For Hadoop} edition.
+ */
+public interface IgniteFileSystem extends IgniteAsyncSupport {
+    /** IGFS scheme name. */
+    public static final String IGFS_SCHEME = "igfs";
+
+    /**
+     * Gets IGFS name.
+     *
+     * @return IGFS name, or {@code null} for default file system.
+     */
+    public String name();
+
+    /**
+     * Gets IGFS configuration.
+     *
+     * @return IGFS configuration.
+     */
+    public FileSystemConfiguration configuration();
+
+    /**
+     * Gets summary (total number of files, total number of directories and total length)
+     * for a given path.
+     *
+     * @param path Path to get information for.
+     * @return Summary object.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path is not found.
+     * @throws IgniteException If failed.
+     */
+    public IgfsPathSummary summary(IgfsPath path) throws IgniteException;
+
+    /**
+     * Opens a file for reading.
+     *
+     * @param path File path to read.
+     * @return File input stream to read data from.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
+     */
+    public IgfsInputStream open(IgfsPath path) throws IgniteException;
+
+    /**
+     * Opens a file for reading.
+     *
+     * @param path File path to read.
+     * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
+     * @return File input stream to read data from.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
+     */
+    public IgfsInputStream open(IgfsPath path, int bufSize) throws IgniteException;
+
+    /**
+     * Opens a file for reading.
+     *
+     * @param path File path to read.
+     * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
+     * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is started.
+     * @return File input stream to read data from.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
+     */
+    public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteException;
+
+    /**
+     * Creates a file and opens it for writing.
+     *
+     * @param path File path to create.
+     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
+     * @return File output stream to write data to.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsOutputStream create(IgfsPath path, boolean overwrite) throws IgniteException;
+
+    /**
+     * Creates a file and opens it for writing.
+     *
+     * @param path File path to create.
+     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
+     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
+     * @param replication Replication factor.
+     * @param blockSize Block size.
+     * @param props File properties to set.
+     * @return File output stream to write data to.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
+        long blockSize, @Nullable Map<String, String> props) throws IgniteException;
+
+    /**
+     * Creates a file and opens it for writing.
+     *
+     * @param path File path to create.
+     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
+     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
+     * @param affKey Affinity key used to store file blocks. If not {@code null}, the whole file will be
+     *      stored on node where {@code affKey} resides.
+     * @param replication Replication factor.
+     * @param blockSize Block size.
+     * @param props File properties to set.
+     * @return File output stream to write data to.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite,
+        @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props)
+        throws IgniteException;
+
+    /**
+     * Opens an output stream to an existing file for appending data.
+     *
+     * @param path File path to append.
+     * @param create Create file if it doesn't exist yet.
+     * @return File output stream to append data to.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist and create flag is {@code false}.
+     */
+    public IgfsOutputStream append(IgfsPath path, boolean create) throws IgniteException;
+
+    /**
+     * Opens an output stream to an existing file for appending data.
+     *
+     * @param path File path to append.
+     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
+     * @param create Create file if it doesn't exist yet.
+     * @param props File properties to set only in case it file was just created.
+     * @return File output stream to append data to.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist and create flag is {@code false}.
+     */
+    public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create, @Nullable Map<String, String> props)
+        throws IgniteException;
+
+    /**
+     * Sets last access time and last modification time for a given path. If argument is {@code null},
+     * corresponding time will not be changed.
+     *
+     * @param path Path to update.
+     * @param accessTime Optional last access time to set. Value {@code -1} does not update access time.
+     * @param modificationTime Optional last modification time to set. Value {@code -1} does not update
+     *      modification time.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If target was not found.
+     * @throws IgniteException If error occurred.
+     */
+    public void setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteException;
+
+    /**
+     * Gets affinity block locations for data blocks of the file, i.e. the nodes, on which the blocks
+     * are stored.
+     *
+     * @param path File path to get affinity for.
+     * @param start Position in the file to start affinity resolution from.
+     * @param len Size of data in the file to resolve affinity for.
+     * @return Affinity block locations.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) throws IgniteException;
+
+    /**
+     * Get affinity block locations for data blocks of the file. In case {@code maxLen} parameter is set and
+     * particular block location length is greater than this value, block locations will be split into smaller
+     * chunks.
+     *
+     * @param path File path to get affinity for.
+     * @param start Position in the file to start affinity resolution from.
+     * @param len Size of data in the file to resolve affinity for.
+     * @param maxLen Maximum length of a single returned block location length.
+     * @return Affinity block locations.
+     * @throws IgniteException In case of error.
+     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len, long maxLen)
+        throws IgniteException;
+
+    /**
+     * Gets metrics snapshot for this file system.
+     *
+     * @return Metrics.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsMetrics metrics() throws IgniteException;
+
+    /**
+     * Resets metrics for this file system.
+     *
+     * @throws IgniteException In case of error.
+     */
+    public void resetMetrics() throws IgniteException;
+
+    /**
+     * Determines size of the file denoted by provided path. In case if path is a directory, then
+     * total size of all containing entries will be calculated recursively.
+     *
+     * @param path File system path.
+     * @return Total size.
+     * @throws IgniteException In case of error.
+     */
+    public long size(IgfsPath path) throws IgniteException;
+
+    /**
+     * Formats the file system removing all existing entries from it.
+     * <p>
+     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
+     *
+     * @throws IgniteException In case format has failed.
+     */
+    @IgniteAsyncSupported
+    public void format() throws IgniteException;
+
+    /**
+     * Executes IGFS task.
+     * <p>
+     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
+     *
+     * @param task Task to execute.
+     * @param rslvr Optional resolver to control split boundaries.
+     * @param paths Collection of paths to be processed within this task.
+     * @param arg Optional task argument.
+     * @return Task result.
+     * @throws IgniteException If execution failed.
+     */
+    @IgniteAsyncSupported
+    public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
+        Collection<IgfsPath> paths, @Nullable T arg) throws IgniteException;
+
+    /**
+     * Executes IGFS task with overridden maximum range length (see
+     * {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()} for more information).
+     * <p>
+     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
+     *
+     * @param task Task to execute.
+     * @param rslvr Optional resolver to control split boundaries.
+     * @param paths Collection of paths to be processed within this task.
+     * @param skipNonExistentFiles Whether to skip non existent files. If set to {@code true} non-existent files will
+     *     be ignored. Otherwise an exception will be thrown.
+     * @param maxRangeLen Optional maximum range length. If {@code 0}, then by default all consecutive
+     *      IGFS blocks will be included.
+     * @param arg Optional task argument.
+     * @return Task result.
+     * @throws IgniteException If execution failed.
+     */
+    @IgniteAsyncSupported
+    public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
+        Collection<IgfsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg)
+        throws IgniteException;
+
+    /**
+     * Executes IGFS task.
+     * <p>
+     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
+     *
+     * @param taskCls Task class to execute.
+     * @param rslvr Optional resolver to control split boundaries.
+     * @param paths Collection of paths to be processed within this task.
+     * @param arg Optional task argument.
+     * @return Task result.
+     * @throws IgniteException If execution failed.
+     */
+    @IgniteAsyncSupported
+    public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
+        @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, @Nullable T arg) throws IgniteException;
+
+    /**
+     * Executes IGFS task with overridden maximum range length (see
+     * {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()} for more information).
+     * <p>
+     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
+     *
+     * @param taskCls Task class to execute.
+     * @param rslvr Optional resolver to control split boundaries.
+     * @param paths Collection of paths to be processed within this task.
+     * @param skipNonExistentFiles Whether to skip non existent files. If set to {@code true} non-existent files will
+     *     be ignored. Otherwise an exception will be thrown.
+     * @param maxRangeLen Maximum range length.
+     * @param arg Optional task argument.
+     * @return Task result.
+     * @throws IgniteException If execution failed.
+     */
+    @IgniteAsyncSupported
+    public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
+        @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, boolean skipNonExistentFiles,
+        long maxRangeLen, @Nullable T arg) throws IgniteException;
+
+    /**
+     * Checks if the specified path exists in the file system.
+     *
+     * @param path Path to check for existence in the file system.
+     * @return {@code True} if such file exists, otherwise - {@code false}.
+     * @throws IgniteException In case of error.
+     */
+    public boolean exists(IgfsPath path);
+
+    /**
+     * Updates file information for the specified path. Existent properties, not listed in the passed collection,
+     * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values
+     * will be removed from the stored properties or ignored if they don't exist in the file info.
+     * <p>
+     * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated
+     * to the secondary file system:
+     * <ul>
+     * <li>{@code usrName} - file owner name;</li>
+     * <li>{@code grpName} - file owner group;</li>
+     * <li>{@code permission} - Unix-style string representing file permissions.</li>
+     * </ul>
+     *
+     * @param path File path to set properties for.
+     * @param props Properties to update.
+     * @return File information for specified path or {@code null} if such path does not exist.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteException;
+
+    /**
+     * Renames/moves a file.
+     * <p>
+     * You are free to rename/move data files as you wish, but directories can be only renamed.
+     * You cannot move the directory between different parent directories.
+     * <p>
+     * Examples:
+     * <ul>
+     *     <li>"/work/file.txt" => "/home/project/Presentation Scenario.txt"</li>
+     *     <li>"/work" => "/work-2012.bkp"</li>
+     *     <li>"/work" => "<strike>/backups/work</strike>" - such operation is restricted for directories.</li>
+     * </ul>
+     *
+     * @param src Source file path to rename.
+     * @param dest Destination file path. If destination path is a directory, then source file will be placed
+     *     into destination directory with original name.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If source file doesn't exist.
+     */
+    public void rename(IgfsPath src, IgfsPath dest) throws IgniteException;
+
+    /**
+     * Deletes file.
+     *
+     * @param path File path to delete.
+     * @param recursive Delete non-empty directories recursively.
+     * @return {@code True} in case of success, {@code false} otherwise.
+     * @throws IgniteException In case of error.
+     */
+    public boolean delete(IgfsPath path, boolean recursive) throws IgniteException;
+
+    /**
+     * Creates directories under specified path.
+     *
+     * @param path Path of directories chain to create.
+     * @throws IgniteException In case of error.
+     */
+    public void mkdirs(IgfsPath path) throws IgniteException;
+
+    /**
+     * Creates directories under specified path with the specified properties.
+     *
+     * @param path Path of directories chain to create.
+     * @param props Metadata properties to set on created directories.
+     * @throws IgniteException In case of error.
+     */
+    public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) throws IgniteException;
+
+    /**
+     * Lists file paths under the specified path.
+     *
+     * @param path Path to list files under.
+     * @return List of files under the specified path.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteException;
+
+    /**
+     * Lists files under the specified path.
+     *
+     * @param path Path to list files under.
+     * @return List of files under the specified path.
+     * @throws IgniteException In case of error.
+     * @throws IgfsFileNotFoundException If path doesn't exist.
+     */
+    public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteException;
+
+    /**
+     * Gets file information for the specified path.
+     *
+     * @param path Path to get information for.
+     * @return File information for specified path or {@code null} if such path does not exist.
+     * @throws IgniteException In case of error.
+     */
+    public IgfsFile info(IgfsPath path) throws IgniteException;
+
+    /**
+     * Gets used space in bytes.
+     *
+     * @return Used space in bytes.
+     * @throws IgniteException In case of error.
+     */
+    public long usedSpaceSize() throws IgniteException;
+
+    /** {@inheritDoc} */
+    @Override public IgniteFileSystem withAsync();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFs.java b/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
deleted file mode 100644
index dc11973..0000000
--- a/modules/core/src/main/java/org/apache/ignite/IgniteFs.java
+++ /dev/null
@@ -1,346 +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;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.igfs.mapreduce.*;
-import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * <b>IG</b>nite <b>F</b>ile <b>S</b>ystem API. It provides a typical file system "view" on a particular cache:
- * <ul>
- *     <li>list directories or get information for a single path</li>
- *     <li>create/move/delete files or directories</li>
- *     <li>write/read data streams into/from files</li>
- * </ul>
- * The data of each file is split on separate data blocks and stored in the cache.
- * You can access file's data with a standard Java streaming API. Moreover, for each part
- * of the file you can calculate an affinity and process file's content on corresponding
- * nodes to escape unnecessary networking.
- * <p/>
- * This API is fully thread-safe and you can use it from several threads.
- * <h1 class="header">IGFS Configuration</h1>
- * The simplest way to run a Ignite node with configured file system is to pass
- * special configuration file included in Ignite distribution to {@code ignite.sh} or
- * {@code ignite.bat} scripts, like this: {@code ignite.sh config/hadoop/default-config.xml}
- * <p>
- * {@code IGFS} can be started as a data node or as a client node. Data node is responsible for
- * caching data, while client node is responsible for basic file system operations and accessing
- * data nodes remotely. When used as Hadoop file system, clients nodes usually started together
- * with {@code job-submitter} or {@code job-scheduler} processes, while data nodes are usually
- * started together with Hadoop {@code task-tracker} processes.
- * <h1 class="header">Integration With Hadoop</h1>
- * In addition to direct file system API, {@code IGFS} can be integrated with {@code Hadoop} by
- * plugging in as {@code Hadoop FileSystem}. Refer to
- * {@code org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem} or
- * {@code org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem} for more information.
- * <p>
- * <b>NOTE:</b> integration with Hadoop is available only in {@code In-Memory Accelerator For Hadoop} edition.
- */
-public interface IgniteFs extends Igfs, IgniteAsyncSupport {
-    /** IGFS scheme name. */
-    public static final String IGFS_SCHEME = "igfs";
-
-    /** File property: prefer writes to local node. */
-    public static final String PROP_PREFER_LOCAL_WRITES = "locWrite";
-
-    /**
-     * Gets IGFS name.
-     *
-     * @return IGFS name, or {@code null} for default file system.
-     */
-    public String name();
-
-    /**
-     * Gets IGFS configuration.
-     *
-     * @return IGFS configuration.
-     */
-    public IgfsConfiguration configuration();
-
-    /**
-     * Gets summary (total number of files, total number of directories and total length)
-     * for a given path.
-     *
-     * @param path Path to get information for.
-     * @return Summary object.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path is not found.
-     * @throws IgniteException If failed.
-     */
-    public IgfsPathSummary summary(IgfsPath path) throws IgniteException;
-
-    /**
-     * Opens a file for reading.
-     *
-     * @param path File path to read.
-     * @return File input stream to read data from.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
-     */
-    public IgfsInputStream open(IgfsPath path) throws IgniteException;
-
-    /**
-     * Opens a file for reading.
-     *
-     * @param path File path to read.
-     * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
-     * @return File input stream to read data from.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
-     */
-    @Override public IgfsInputStream open(IgfsPath path, int bufSize) throws IgniteException;
-
-    /**
-     * Opens a file for reading.
-     *
-     * @param path File path to read.
-     * @param bufSize Read buffer size (bytes) or {@code zero} to use default value.
-     * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is started.
-     * @return File input stream to read data from.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
-     */
-    public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteException;
-
-    /**
-     * Creates a file and opens it for writing.
-     *
-     * @param path File path to create.
-     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
-     * @return File output stream to write data to.
-     * @throws IgniteException In case of error.
-     */
-    @Override public IgfsOutputStream create(IgfsPath path, boolean overwrite) throws IgniteException;
-
-    /**
-     * Creates a file and opens it for writing.
-     *
-     * @param path File path to create.
-     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
-     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
-     * @param replication Replication factor.
-     * @param blockSize Block size.
-     * @param props File properties to set.
-     * @return File output stream to write data to.
-     * @throws IgniteException In case of error.
-     */
-    @Override public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
-        long blockSize, @Nullable Map<String, String> props) throws IgniteException;
-
-    /**
-     * Creates a file and opens it for writing.
-     *
-     * @param path File path to create.
-     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
-     * @param overwrite Overwrite file if it already exists. Note: you cannot overwrite an existent directory.
-     * @param affKey Affinity key used to store file blocks. If not {@code null}, the whole file will be
-     *      stored on node where {@code affKey} resides.
-     * @param replication Replication factor.
-     * @param blockSize Block size.
-     * @param props File properties to set.
-     * @return File output stream to write data to.
-     * @throws IgniteException In case of error.
-     */
-    public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite,
-        @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props)
-        throws IgniteException;
-
-    /**
-     * Opens an output stream to an existing file for appending data.
-     *
-     * @param path File path to append.
-     * @param create Create file if it doesn't exist yet.
-     * @return File output stream to append data to.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist and create flag is {@code false}.
-     */
-    public IgfsOutputStream append(IgfsPath path, boolean create) throws IgniteException;
-
-    /**
-     * Opens an output stream to an existing file for appending data.
-     *
-     * @param path File path to append.
-     * @param bufSize Write buffer size (bytes) or {@code zero} to use default value.
-     * @param create Create file if it doesn't exist yet.
-     * @param props File properties to set only in case it file was just created.
-     * @return File output stream to append data to.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist and create flag is {@code false}.
-     */
-    @Override public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) throws IgniteException;
-
-    /**
-     * Sets last access time and last modification time for a given path. If argument is {@code null},
-     * corresponding time will not be changed.
-     *
-     * @param path Path to update.
-     * @param accessTime Optional last access time to set. Value {@code -1} does not update access time.
-     * @param modificationTime Optional last modification time to set. Value {@code -1} does not update
-     *      modification time.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If target was not found.
-     * @throws IgniteException If error occurred.
-     */
-    public void setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteException;
-
-    /**
-     * Gets affinity block locations for data blocks of the file, i.e. the nodes, on which the blocks
-     * are stored.
-     *
-     * @param path File path to get affinity for.
-     * @param start Position in the file to start affinity resolution from.
-     * @param len Size of data in the file to resolve affinity for.
-     * @return Affinity block locations.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
-     */
-    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) throws IgniteException;
-
-    /**
-     * Get affinity block locations for data blocks of the file. In case {@code maxLen} parameter is set and
-     * particular block location length is greater than this value, block locations will be split into smaller
-     * chunks.
-     *
-     * @param path File path to get affinity for.
-     * @param start Position in the file to start affinity resolution from.
-     * @param len Size of data in the file to resolve affinity for.
-     * @param maxLen Maximum length of a single returned block location length.
-     * @return Affinity block locations.
-     * @throws IgniteException In case of error.
-     * @throws org.apache.ignite.igfs.IgfsFileNotFoundException If path doesn't exist.
-     */
-    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len, long maxLen)
-        throws IgniteException;
-
-    /**
-     * Gets metrics snapshot for this file system.
-     *
-     * @return Metrics.
-     * @throws IgniteException In case of error.
-     */
-    public IgfsMetrics metrics() throws IgniteException;
-
-    /**
-     * Resets metrics for this file system.
-     *
-     * @throws IgniteException In case of error.
-     */
-    public void resetMetrics() throws IgniteException;
-
-    /**
-     * Determines size of the file denoted by provided path. In case if path is a directory, then
-     * total size of all containing entries will be calculated recursively.
-     *
-     * @param path File system path.
-     * @return Total size.
-     * @throws IgniteException In case of error.
-     */
-    public long size(IgfsPath path) throws IgniteException;
-
-    /**
-     * Formats the file system removing all existing entries from it.
-     * <p>
-     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
-     *
-     * @throws IgniteException In case format has failed.
-     */
-    @IgniteAsyncSupported
-    public void format() throws IgniteException;
-
-    /**
-     * Executes IGFS task.
-     * <p>
-     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
-     *
-     * @param task Task to execute.
-     * @param rslvr Optional resolver to control split boundaries.
-     * @param paths Collection of paths to be processed within this task.
-     * @param arg Optional task argument.
-     * @return Task result.
-     * @throws IgniteException If execution failed.
-     */
-    @IgniteAsyncSupported
-    public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
-        Collection<IgfsPath> paths, @Nullable T arg) throws IgniteException;
-
-    /**
-     * Executes IGFS task with overridden maximum range length (see
-     * {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()} for more information).
-     * <p>
-     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
-     *
-     * @param task Task to execute.
-     * @param rslvr Optional resolver to control split boundaries.
-     * @param paths Collection of paths to be processed within this task.
-     * @param skipNonExistentFiles Whether to skip non existent files. If set to {@code true} non-existent files will
-     *     be ignored. Otherwise an exception will be thrown.
-     * @param maxRangeLen Optional maximum range length. If {@code 0}, then by default all consecutive
-     *      IGFS blocks will be included.
-     * @param arg Optional task argument.
-     * @return Task result.
-     * @throws IgniteException If execution failed.
-     */
-    @IgniteAsyncSupported
-    public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
-        Collection<IgfsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg)
-        throws IgniteException;
-
-    /**
-     * Executes IGFS task.
-     * <p>
-     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
-     *
-     * @param taskCls Task class to execute.
-     * @param rslvr Optional resolver to control split boundaries.
-     * @param paths Collection of paths to be processed within this task.
-     * @param arg Optional task argument.
-     * @return Task result.
-     * @throws IgniteException If execution failed.
-     */
-    @IgniteAsyncSupported
-    public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
-        @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, @Nullable T arg) throws IgniteException;
-
-    /**
-     * Executes IGFS task with overridden maximum range length (see
-     * {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()} for more information).
-     * <p>
-     * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
-     *
-     * @param taskCls Task class to execute.
-     * @param rslvr Optional resolver to control split boundaries.
-     * @param paths Collection of paths to be processed within this task.
-     * @param skipNonExistentFiles Whether to skip non existent files. If set to {@code true} non-existent files will
-     *     be ignored. Otherwise an exception will be thrown.
-     * @param maxRangeLen Maximum range length.
-     * @param arg Optional task argument.
-     * @return Task result.
-     * @throws IgniteException If execution failed.
-     */
-    @IgniteAsyncSupported
-    public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
-        @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, boolean skipNonExistentFiles,
-        long maxRangeLen, @Nullable T arg) throws IgniteException;
-
-    /** {@inheritDoc} */
-    @Override public IgniteFs withAsync();
-}


[52/58] [abbrv] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Posted by yz...@apache.org.
Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-futures-cleanup-1

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
	modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java


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

Branch: refs/heads/ignite-313
Commit: 17ac3602c3ea9afd1efc8ebefc06009cdcd87bb9
Parents: bc18635 5d06523
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Mar 5 11:16:46 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Mar 5 11:16:46 2015 +0300

----------------------------------------------------------------------
 bin/ignitevisorcmd.sh                           |    2 +-
 bin/setup-hadoop.bat                            |    2 +-
 bin/setup-hadoop.sh                             |    2 +-
 config/hadoop/default-config.xml                |   10 +-
 docs/core-site.ignite.xml                       |    6 +-
 examples/config/filesystem/core-site.xml        |    4 +-
 examples/config/filesystem/example-igfs.xml     |    4 +-
 .../ignite/examples/igfs/IgfsExample.java       |   18 +-
 .../examples/igfs/IgfsMapReduceExample.java     |    6 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   13 +-
 .../org/apache/ignite/IgniteFileSystem.java     |  457 ++++
 .../main/java/org/apache/ignite/IgniteFs.java   |  346 ---
 .../configuration/FileSystemConfiguration.java  |  805 ++++++
 .../configuration/HadoopConfiguration.java      |  173 ++
 .../ignite/configuration/IgfsConfiguration.java |  807 ------
 .../configuration/IgniteConfiguration.java      |   19 +-
 .../main/java/org/apache/ignite/igfs/Igfs.java  |  216 --
 .../apache/ignite/igfs/IgfsBlockLocation.java   |    2 +-
 .../java/org/apache/ignite/igfs/IgfsFile.java   |    2 +-
 .../org/apache/ignite/igfs/IgfsInputStream.java |    6 +-
 .../org/apache/ignite/igfs/IgfsMetrics.java     |    2 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    8 +-
 .../java/org/apache/ignite/igfs/IgfsReader.java |   38 -
 .../mapreduce/IgfsInputStreamJobAdapter.java    |    4 +-
 .../apache/ignite/igfs/mapreduce/IgfsJob.java   |    4 +-
 .../igfs/mapreduce/IgfsRecordResolver.java      |    2 +-
 .../apache/ignite/igfs/mapreduce/IgfsTask.java  |    4 +-
 .../ignite/igfs/mapreduce/IgfsTaskArgs.java     |    2 +-
 .../IgfsByteDelimiterRecordResolver.java        |    3 +-
 .../records/IgfsFixedLengthRecordResolver.java  |    2 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  201 ++
 ...fsSecondaryFileSystemPositionedReadable.java |   38 +
 .../apache/ignite/igfs/secondary/package.html   |   24 +
 .../ignite/internal/GridKernalContext.java      |    2 +-
 .../ignite/internal/GridKernalContextImpl.java  |    8 +-
 .../ignite/internal/IgniteComponentType.java    |    4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |    4 +-
 .../apache/ignite/internal/IgniteKernal.java    |   14 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    8 +-
 .../processors/cache/GridCacheAdapter.java      |    4 +-
 .../processors/cache/GridCacheProcessor.java    |    4 +-
 .../processors/cache/GridCacheUtils.java        |    4 +-
 .../internal/processors/hadoop/GridHadoop.java  |   86 -
 .../hadoop/GridHadoopConfiguration.java         |  172 --
 .../processors/hadoop/GridHadoopCounter.java    |   44 -
 .../hadoop/GridHadoopCounterWriter.java         |   36 -
 .../processors/hadoop/GridHadoopCounters.java   |   49 -
 .../processors/hadoop/GridHadoopFileBlock.java  |  162 --
 .../processors/hadoop/GridHadoopInputSplit.java |   54 -
 .../processors/hadoop/GridHadoopJob.java        |  102 -
 .../processors/hadoop/GridHadoopJobId.java      |  103 -
 .../processors/hadoop/GridHadoopJobInfo.java    |   83 -
 .../processors/hadoop/GridHadoopJobPhase.java   |   38 -
 .../hadoop/GridHadoopJobProperty.java           |  138 -
 .../processors/hadoop/GridHadoopJobStatus.java  |  207 --
 .../hadoop/GridHadoopMapReducePlan.java         |   80 -
 .../hadoop/GridHadoopMapReducePlanner.java      |   40 -
 .../hadoop/GridHadoopPartitioner.java           |   33 -
 .../hadoop/GridHadoopSerialization.java         |   54 -
 .../processors/hadoop/GridHadoopTask.java       |   72 -
 .../hadoop/GridHadoopTaskContext.java           |  189 --
 .../processors/hadoop/GridHadoopTaskInfo.java   |  153 --
 .../processors/hadoop/GridHadoopTaskInput.java  |   55 -
 .../processors/hadoop/GridHadoopTaskOutput.java |   40 -
 .../processors/hadoop/GridHadoopTaskType.java   |   56 -
 .../internal/processors/hadoop/Hadoop.java      |   88 +
 .../processors/hadoop/HadoopFileBlock.java      |  162 ++
 .../processors/hadoop/HadoopInputSplit.java     |   54 +
 .../internal/processors/hadoop/HadoopJob.java   |  102 +
 .../internal/processors/hadoop/HadoopJobId.java |  103 +
 .../processors/hadoop/HadoopJobInfo.java        |   83 +
 .../processors/hadoop/HadoopJobPhase.java       |   38 +
 .../processors/hadoop/HadoopJobProperty.java    |  138 +
 .../processors/hadoop/HadoopJobStatus.java      |  207 ++
 .../processors/hadoop/HadoopMapReducePlan.java  |   80 +
 .../hadoop/HadoopMapReducePlanner.java          |   40 +
 .../processors/hadoop/HadoopNoopProcessor.java  |   76 +
 .../processors/hadoop/HadoopPartitioner.java    |   33 +
 .../hadoop/HadoopProcessorAdapter.java          |   96 +
 .../processors/hadoop/HadoopSerialization.java  |   54 +
 .../internal/processors/hadoop/HadoopTask.java  |   72 +
 .../processors/hadoop/HadoopTaskContext.java    |  190 ++
 .../processors/hadoop/HadoopTaskInfo.java       |  153 ++
 .../processors/hadoop/HadoopTaskInput.java      |   55 +
 .../processors/hadoop/HadoopTaskOutput.java     |   40 +
 .../processors/hadoop/HadoopTaskType.java       |   56 +
 .../hadoop/IgniteHadoopNoopProcessor.java       |   74 -
 .../hadoop/IgniteHadoopProcessorAdapter.java    |   94 -
 .../hadoop/counter/HadoopCounter.java           |   44 +
 .../hadoop/counter/HadoopCounterWriter.java     |   37 +
 .../hadoop/counter/HadoopCounters.java          |   49 +
 .../internal/processors/igfs/IgfsAsyncImpl.java |    9 +-
 .../internal/processors/igfs/IgfsContext.java   |    6 +-
 .../processors/igfs/IgfsDataManager.java        |    5 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   28 +-
 .../internal/processors/igfs/IgfsFileInfo.java  |    4 +-
 .../internal/processors/igfs/IgfsImpl.java      |   42 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |    4 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |    5 +-
 .../internal/processors/igfs/IgfsJobImpl.java   |    2 +-
 .../processors/igfs/IgfsMetaManager.java        |   27 +-
 .../processors/igfs/IgfsNoopProcessor.java      |    4 +-
 .../processors/igfs/IgfsOutputStreamImpl.java   |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |   26 +-
 .../processors/igfs/IgfsProcessorAdapter.java   |    4 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |  121 +
 .../IgfsSecondaryInputStreamDescriptor.java     |    8 +-
 .../processors/igfs/IgfsServerManager.java      |    4 +-
 .../ignite/internal/util/IgniteUtils.java       |   61 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    3 +-
 .../ignite/internal/visor/igfs/VisorIgfs.java   |    4 +-
 .../visor/igfs/VisorIgfsProfilerClearTask.java  |    2 +-
 .../visor/node/VisorGridConfiguration.java      |    2 +-
 .../visor/node/VisorIgfsConfiguration.java      |    9 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    2 +-
 .../internal/visor/util/VisorTaskUtils.java     |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../hadoop/core-site-loopback-secondary.xml     |    4 +-
 .../test/config/hadoop/core-site-loopback.xml   |    4 +-
 .../test/config/hadoop/core-site-secondary.xml  |    4 +-
 .../core/src/test/config/hadoop/core-site.xml   |    4 +-
 modules/core/src/test/config/igfs-loopback.xml  |    4 +-
 modules/core/src/test/config/igfs-shmem.xml     |    4 +-
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   10 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |    4 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |    8 +-
 .../igfs/IgfsFragmentizerTopologySelfTest.java  |    2 +-
 ...heIgfsPerBlockLruEvictionPolicySelfTest.java |  485 ----
 .../processors/igfs/IgfsAbstractSelfTest.java   |   85 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |  485 ++++
 .../processors/igfs/IgfsCacheSelfTest.java      |    4 +-
 .../igfs/IgfsDataManagerSelfTest.java           |    5 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   12 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |    4 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   22 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   14 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   10 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |   50 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |   10 +-
 ...dpointRegistrationOnLinuxAndMacSelfTest.java |    4 +-
 ...pcEndpointRegistrationOnWindowsSelfTest.java |    2 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |    4 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   14 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   10 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |    6 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |    4 +-
 .../ignite/testframework/junits/IgniteMock.java |    4 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    6 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    2 +-
 .../client/hadoop/GridHadoopClientProtocol.java |  334 ---
 .../GridHadoopClientProtocolProvider.java       |  137 -
 .../counter/GridHadoopClientCounterGroup.java   |  121 -
 .../counter/GridHadoopClientCounters.java       |  217 --
 .../apache/ignite/client/hadoop/package.html    |   24 -
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   93 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  414 +++
 .../org/apache/ignite/hadoop/fs/package.html    |   24 +
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 1237 +++++++++
 .../org/apache/ignite/hadoop/fs/v1/package.html |   24 +
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  982 ++++++++
 .../org/apache/ignite/hadoop/fs/v2/package.html |   24 +
 .../IgniteHadoopClientProtocolProvider.java     |  138 +
 .../mapreduce/IgniteHadoopMapReducePlanner.java |  435 ++++
 .../apache/ignite/hadoop/mapreduce/package.html |   24 +
 .../java/org/apache/ignite/hadoop/package.html  |   24 +
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  414 ---
 .../igfs/hadoop/IgfsHadoopParameters.java       |   94 -
 .../org/apache/ignite/igfs/hadoop/package.html  |   24 -
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    | 1234 ---------
 .../apache/ignite/igfs/hadoop/v1/package.html   |   24 -
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |  982 --------
 .../apache/ignite/igfs/hadoop/v2/package.html   |   24 -
 .../java/org/apache/ignite/igfs/package.html    |   24 -
 .../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 ----
 .../ignite/internal/igfs/hadoop/package.html    |   24 -
 .../apache/ignite/internal/igfs/package.html    |   24 -
 .../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        |  197 ++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  163 ++
 .../internal/processors/hadoop/HadoopImpl.java  |  134 +
 .../hadoop/HadoopMapReduceCounterGroup.java     |  121 +
 .../hadoop/HadoopMapReduceCounters.java         |  216 ++
 .../processors/hadoop/HadoopProcessor.java      |  227 ++
 .../internal/processors/hadoop/HadoopSetup.java |  505 ++++
 .../hadoop/HadoopTaskCancelledException.java    |   35 +
 .../internal/processors/hadoop/HadoopUtils.java |  308 +++
 .../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    |  127 +
 .../hadoop/counter/HadoopCountersImpl.java      |  197 ++
 .../hadoop/counter/HadoopLongCounter.java       |   90 +
 .../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 +
 .../processors/hadoop/fs/HadoopParameters.java  |   94 +
 .../hadoop/fs/HadoopRawLocalFileSystem.java     |  304 +++
 .../processors/hadoop/igfs/HadoopIgfs.java      |  198 ++
 .../igfs/HadoopIgfsCommunicationException.java  |   57 +
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |  210 ++
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |   88 +
 .../hadoop/igfs/HadoopIgfsFuture.java           |   94 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  409 +++
 .../hadoop/igfs/HadoopIgfsInputStream.java      |  626 +++++
 .../processors/hadoop/igfs/HadoopIgfsIo.java    |   76 +
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |  599 +++++
 .../hadoop/igfs/HadoopIgfsIpcIoListener.java    |   36 +
 .../hadoop/igfs/HadoopIgfsJclLogger.java        |  115 +
 .../hadoop/igfs/HadoopIgfsOutProc.java          |  466 ++++
 .../hadoop/igfs/HadoopIgfsOutputStream.java     |  201 ++
 .../hadoop/igfs/HadoopIgfsProperties.java       |   88 +
 .../hadoop/igfs/HadoopIgfsProxyInputStream.java |  335 +++
 .../igfs/HadoopIgfsProxyOutputStream.java       |  165 ++
 ...fsSecondaryFileSystemPositionedReadable.java |  104 +
 .../hadoop/igfs/HadoopIgfsStreamDelegate.java   |   96 +
 .../igfs/HadoopIgfsStreamEventListener.java     |   39 +
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |  131 +
 .../hadoop/igfs/HadoopIgfsWrapper.java          |  511 ++++
 .../jobtracker/GridHadoopJobMetadata.java       |  305 ---
 .../hadoop/jobtracker/GridHadoopJobTracker.java | 1625 ------------
 .../hadoop/jobtracker/HadoopJobMetadata.java    |  306 +++
 .../hadoop/jobtracker/HadoopJobTracker.java     | 1626 ++++++++++++
 .../hadoop/message/GridHadoopMessage.java       |   27 -
 .../hadoop/message/HadoopMessage.java           |   27 +
 .../planner/GridHadoopDefaultMapReducePlan.java |  107 -
 .../GridHadoopDefaultMapReducePlanner.java      |  434 ----
 .../planner/HadoopDefaultMapReducePlan.java     |  107 +
 .../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      |  333 +++
 .../proto/HadoopProtocolJobCountersTask.java    |   46 +
 .../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/GridHadoopShuffle.java       |  256 --
 .../hadoop/shuffle/GridHadoopShuffleAck.java    |   91 -
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |  593 -----
 .../shuffle/GridHadoopShuffleMessage.java       |  242 --
 .../hadoop/shuffle/HadoopShuffle.java           |  256 ++
 .../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 +
 .../GridHadoopEmbeddedTaskExecutor.java         |  146 --
 .../taskexecutor/GridHadoopExecutorService.java |  232 --
 .../taskexecutor/GridHadoopRunnableTask.java    |  268 --
 .../GridHadoopTaskExecutorAdapter.java          |   57 -
 .../taskexecutor/GridHadoopTaskState.java       |   38 -
 .../taskexecutor/GridHadoopTaskStatus.java      |  114 -
 .../HadoopEmbeddedTaskExecutor.java             |  146 ++
 .../taskexecutor/HadoopExecutorService.java     |  231 ++
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  268 ++
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   57 +
 .../hadoop/taskexecutor/HadoopTaskState.java    |   38 +
 .../hadoop/taskexecutor/HadoopTaskStatus.java   |  114 +
 .../GridHadoopExternalTaskExecutor.java         |  960 -------
 .../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    |  958 +++++++
 .../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     |  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          |  444 ++++
 .../hadoop/v2/HadoopWritableSerialization.java  |   74 +
 ...op.mapreduce.protocol.ClientProtocolProvider |    2 +-
 ...ridHadoopClientProtocolEmbeddedSelfTest.java |   34 -
 .../GridHadoopClientProtocolSelfTest.java       |  633 -----
 .../HadoopClientProtocolEmbeddedSelfTest.java   |   34 +
 .../hadoop/HadoopClientProtocolSelfTest.java    |  635 +++++
 .../HadoopIgfs20FileSystemAbstractSelfTest.java | 1967 +++++++++++++++
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   74 +
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   74 +
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |  305 +++
 .../igfs/HadoopIgfsDualAsyncSelfTest.java       |   32 +
 .../ignite/igfs/HadoopIgfsDualSyncSelfTest.java |   32 +
 ...oopSecondaryFileSystemConfigurationTest.java |   44 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   54 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java | 1967 ---------------
 ...doop20FileSystemLoopbackPrimarySelfTest.java |   74 -
 ...sHadoop20FileSystemShmemPrimarySelfTest.java |   74 -
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |  304 ---
 .../igfs/IgfsHadoopDualAsyncSelfTest.java       |   32 -
 .../ignite/igfs/IgfsHadoopDualSyncSelfTest.java |   32 -
 .../IgfsHadoopFileSystemAbstractSelfTest.java   | 2366 ------------------
 .../IgfsHadoopFileSystemClientSelfTest.java     |  199 --
 .../IgfsHadoopFileSystemHandshakeSelfTest.java  |  311 ---
 .../IgfsHadoopFileSystemIpcCacheSelfTest.java   |  207 --
 .../IgfsHadoopFileSystemLoggerSelfTest.java     |  287 ---
 ...IgfsHadoopFileSystemLoggerStateSelfTest.java |  325 ---
 ...adoopFileSystemLoopbackAbstractSelfTest.java |   46 -
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |   33 -
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |   33 -
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |   33 -
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |   34 -
 ...SystemLoopbackExternalDualAsyncSelfTest.java |   33 -
 ...eSystemLoopbackExternalDualSyncSelfTest.java |   33 -
 ...leSystemLoopbackExternalPrimarySelfTest.java |   33 -
 ...SystemLoopbackExternalSecondarySelfTest.java |   34 -
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |  319 ---
 ...fsHadoopFileSystemShmemAbstractSelfTest.java |   88 -
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 -
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 -
 ...ileSystemShmemExternalDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemExternalDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemExternalPrimarySelfTest.java |   33 -
 ...ileSystemShmemExternalSecondarySelfTest.java |   33 -
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java | 2366 ++++++++++++++++++
 .../IgniteHadoopFileSystemClientSelfTest.java   |  199 ++
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  311 +++
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |  207 ++
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |  287 +++
 ...niteHadoopFileSystemLoggerStateSelfTest.java |  325 +++
 ...adoopFileSystemLoopbackAbstractSelfTest.java |   46 +
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |   33 +
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |   33 +
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |   33 +
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |   34 +
 ...SystemLoopbackExternalDualAsyncSelfTest.java |   33 +
 ...eSystemLoopbackExternalDualSyncSelfTest.java |   33 +
 ...leSystemLoopbackExternalPrimarySelfTest.java |   33 +
 ...SystemLoopbackExternalSecondarySelfTest.java |   34 +
 ...teHadoopFileSystemSecondaryModeSelfTest.java |  319 +++
 ...teHadoopFileSystemShmemAbstractSelfTest.java |   88 +
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 +
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 +
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 +
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 +
 ...ileSystemShmemExternalDualAsyncSelfTest.java |   33 +
 ...FileSystemShmemExternalDualSyncSelfTest.java |   33 +
 ...pFileSystemShmemExternalPrimarySelfTest.java |   33 +
 ...ileSystemShmemExternalSecondarySelfTest.java |   33 +
 .../hadoop/GridHadoopAbstractSelfTest.java      |  222 --
 .../hadoop/GridHadoopAbstractWordCountTest.java |  138 -
 .../hadoop/GridHadoopClassLoaderTest.java       |   69 -
 .../hadoop/GridHadoopCommandLineTest.java       |  440 ----
 ...idHadoopDefaultMapReducePlannerSelfTest.java | 1005 --------
 .../hadoop/GridHadoopFileSystemsTest.java       |  177 --
 .../hadoop/GridHadoopGroupingTest.java          |  286 ---
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |  330 ---
 .../GridHadoopMapReduceEmbeddedSelfTest.java    |  245 --
 .../hadoop/GridHadoopMapReduceTest.java         |  195 --
 .../hadoop/GridHadoopPopularWordsTest.java      |  294 ---
 .../GridHadoopSerializationWrapperSelfTest.java |   74 -
 .../processors/hadoop/GridHadoopSharedMap.java  |   67 -
 .../hadoop/GridHadoopSortingExternalTest.java   |   32 -
 .../hadoop/GridHadoopSortingTest.java           |  281 ---
 .../hadoop/GridHadoopSplitWrapperSelfTest.java  |   68 -
 .../processors/hadoop/GridHadoopStartup.java    |   55 -
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |  551 ----
 .../hadoop/GridHadoopTasksAllVersionsTest.java  |  259 --
 .../hadoop/GridHadoopTasksV1Test.java           |   57 -
 .../hadoop/GridHadoopTasksV2Test.java           |   75 -
 .../GridHadoopTestRoundRobinMrPlanner.java      |   66 -
 .../hadoop/GridHadoopTestTaskContext.java       |  219 --
 .../processors/hadoop/GridHadoopTestUtils.java  |  102 -
 .../hadoop/GridHadoopV2JobSelfTest.java         |   88 -
 .../hadoop/GridHadoopValidationSelfTest.java    |   53 -
 .../hadoop/HadoopAbstractSelfTest.java          |  222 ++
 .../hadoop/HadoopAbstractWordCountTest.java     |  138 +
 .../hadoop/HadoopClassLoaderTest.java           |   69 +
 .../hadoop/HadoopCommandLineTest.java           |  440 ++++
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 1006 ++++++++
 .../hadoop/HadoopFileSystemsTest.java           |  177 ++
 .../processors/hadoop/HadoopGroupingTest.java   |  287 +++
 .../hadoop/HadoopJobTrackerSelfTest.java        |  331 +++
 .../hadoop/HadoopMapReduceEmbeddedSelfTest.java |  246 ++
 .../processors/hadoop/HadoopMapReduceTest.java  |  197 ++
 .../hadoop/HadoopPopularWordsTest.java          |  294 +++
 .../HadoopSerializationWrapperSelfTest.java     |   74 +
 .../processors/hadoop/HadoopSharedMap.java      |   67 +
 .../hadoop/HadoopSortingExternalTest.java       |   34 +
 .../processors/hadoop/HadoopSortingTest.java    |  282 +++
 .../hadoop/HadoopSplitWrapperSelfTest.java      |   68 +
 .../processors/hadoop/HadoopStartup.java        |   54 +
 .../hadoop/HadoopTaskExecutionSelfTest.java     |  551 ++++
 .../hadoop/HadoopTasksAllVersionsTest.java      |  259 ++
 .../processors/hadoop/HadoopTasksV1Test.java    |   57 +
 .../processors/hadoop/HadoopTasksV2Test.java    |   75 +
 .../hadoop/HadoopTestRoundRobinMrPlanner.java   |   66 +
 .../hadoop/HadoopTestTaskContext.java           |  219 ++
 .../processors/hadoop/HadoopTestUtils.java      |  102 +
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   88 +
 .../hadoop/HadoopValidationSelfTest.java        |   53 +
 .../hadoop/examples/GridHadoopWordCount1.java   |   88 -
 .../examples/GridHadoopWordCount1Map.java       |   62 -
 .../examples/GridHadoopWordCount1Reduce.java    |   51 -
 .../hadoop/examples/GridHadoopWordCount2.java   |   95 -
 .../examples/GridHadoopWordCount2Mapper.java    |   72 -
 .../examples/GridHadoopWordCount2Reducer.java   |   70 -
 .../hadoop/examples/HadoopWordCount1.java       |   88 +
 .../hadoop/examples/HadoopWordCount1Map.java    |   62 +
 .../hadoop/examples/HadoopWordCount1Reduce.java |   51 +
 .../hadoop/examples/HadoopWordCount2.java       |   95 +
 .../hadoop/examples/HadoopWordCount2Mapper.java |   72 +
 .../examples/HadoopWordCount2Reducer.java       |   70 +
 .../collections/GridHadoopAbstractMapTest.java  |  152 --
 ...ridHadoopConcurrentHashMultimapSelftest.java |  267 --
 .../collections/GridHadoopHashMapSelfTest.java  |  170 --
 .../collections/GridHadoopSkipListSelfTest.java |  303 ---
 .../collections/HadoopAbstractMapTest.java      |  154 ++
 .../HadoopConcurrentHashMultimapSelftest.java   |  267 ++
 .../collections/HadoopHashMapSelfTest.java      |  170 ++
 .../collections/HadoopSkipListSelfTest.java     |  303 +++
 .../streams/GridHadoopDataStreamSelfTest.java   |  151 --
 .../streams/HadoopDataStreamSelfTest.java       |  151 ++
 .../GridHadoopExecutorServiceTest.java          |  119 -
 .../taskexecutor/HadoopExecutorServiceTest.java |  119 +
 ...GridHadoopExternalTaskExecutionSelfTest.java |  210 --
 .../HadoopExternalTaskExecutionSelfTest.java    |  211 ++
 ...GridHadoopExternalCommunicationSelfTest.java |  209 --
 .../HadoopExternalCommunicationSelfTest.java    |  209 ++
 .../testsuites/IgniteHadoopTestSuite.java       |   82 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   22 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    4 +-
 .../apache/ignite/visor/plugin/VisorPlugin.java |    2 +-
 pom.xml                                         |   12 +-
 567 files changed, 47648 insertions(+), 47299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
index 0000000,eb84d00..3b0f536
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java
@@@ -1,0 -1,76 +1,76 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.ignite.internal.processors.hadoop;
+ 
+ import org.apache.ignite.*;
+ import org.apache.ignite.configuration.*;
+ import org.apache.ignite.internal.*;
+ import org.apache.ignite.internal.processors.hadoop.counter.*;
+ import org.apache.ignite.internal.util.future.*;
+ 
+ /**
+  * Hadoop processor.
+  */
+ public class HadoopNoopProcessor extends HadoopProcessorAdapter {
+     /**
+      * @param ctx Kernal context.
+      */
+     public HadoopNoopProcessor(GridKernalContext ctx) {
+         super(ctx);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Hadoop hadoop() {
+         throw new IllegalStateException("Hadoop module is not found in class path.");
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public HadoopConfiguration config() {
+         return null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public HadoopJobId nextJobId() {
+         return null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteInternalFuture<?> submit(HadoopJobId jobId, HadoopJobInfo jobInfo) {
 -        return new GridFinishedFutureEx<>(new IgniteCheckedException("Hadoop is not available."));
++        return new GridFinishedFuture<>(new IgniteCheckedException("Hadoop is not available."));
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public HadoopJobStatus status(HadoopJobId jobId) throws IgniteCheckedException {
+         return null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public HadoopCounters counters(HadoopJobId jobId) {
+         return null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteInternalFuture<?> finishFuture(HadoopJobId jobId) throws IgniteCheckedException {
+         return null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean kill(HadoopJobId jobId) throws IgniteCheckedException {
+         return false;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ac3602/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------


[23/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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 fd4a030..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 GridHadoopMultimap 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;
-
-        GridHadoopTaskState state = GridHadoopTaskState.COMPLETED;
-
-        GridHadoopPerformanceCounter perfCntr = null;
-
-        try {
-            ctx = job.getTaskContext(info);
-
-            perfCntr = GridHadoopPerformanceCounter.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 (GridHadoopTaskCancelledException ignored) {
-            state = GridHadoopTaskState.CANCELED;
-        }
-        catch (Throwable e) {
-            state = GridHadoopTaskState.FAILED;
-            err = e;
-
-            U.error(log, "Task execution failed.", e);
-        }
-        finally {
-            execEndTs = U.currentTimeMillis();
-
-            if (perfCntr != null)
-                perfCntr.onTaskFinish(info, execEndTs);
-
-            onTaskFinished(new GridHadoopTaskStatus(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(GridHadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
-        if (cancelled)
-            throw new GridHadoopTaskCancelledException("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(GridHadoopTaskStatus 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 GridHadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)):
-                        new GridHadoopSkipList(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/6423cf02/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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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..a3c20d8
--- /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<HadoopJobId, Collection<HadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
+
+    /** Executor service to run tasks. */
+    private HadoopExecutorService exec;
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        super.onKernalStart();
+
+        jobTracker = ctx.jobTracker();
+
+        exec = new HadoopExecutorService(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 (HadoopJobId 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 HadoopJob job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException {
+        if (log.isDebugEnabled())
+            log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() +
+                ", tasksCnt=" + tasks.size() + ']');
+
+        Collection<HadoopRunnableTask> executedTasks = jobs.get(job.id());
+
+        if (executedTasks == null) {
+            executedTasks = new GridConcurrentHashSet<>();
+
+            Collection<HadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks);
+
+            assert extractedCol == null;
+        }
+
+        final Collection<HadoopRunnableTask> finalExecutedTasks = executedTasks;
+
+        for (final HadoopTaskInfo info : tasks) {
+            assert info != null;
+
+            HadoopRunnableTask task = new HadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
+                ctx.localNodeId()) {
+                @Override protected void onTaskFinished(HadoopTaskStatus 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 HadoopTaskInput createInput(HadoopTaskContext taskCtx) throws IgniteCheckedException {
+                    return ctx.shuffle().input(taskCtx);
+                }
+
+                @Override protected HadoopTaskOutput createOutput(HadoopTaskContext 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(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via
+     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called.
+     *
+     * @param jobId Job ID to cancel.
+     */
+    @Override public void cancelTasks(HadoopJobId jobId) {
+        Collection<HadoopRunnableTask> executedTasks = jobs.get(jobId);
+
+        if (executedTasks != null) {
+            for (HadoopRunnableTask task : executedTasks)
+                task.cancel();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException {
+        if (meta.phase() == HadoopJobPhase.PHASE_COMPLETE) {
+            Collection<HadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
+
+            assert executedTasks == null || executedTasks.isEmpty();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..1c318e9
--- /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 HadoopRunnableTask) {
+            final HadoopTaskInfo i = ((HadoopRunnableTask)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/6423cf02/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..2b36267
--- /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.HadoopJobProperty.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.*;
+
+/**
+ * Runnable task.
+ */
+public abstract class HadoopRunnableTask implements Callable<Void> {
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final HadoopJob job;
+
+    /** Task to run. */
+    private final HadoopTaskInfo 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 HadoopTaskContext 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, HadoopJob job, GridUnsafeMemory mem, HadoopTaskInfo 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 HadoopTaskInfo(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 (HadoopTaskOutput out = createOutputInternal(ctx);
+             HadoopTaskInput 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 HadoopTaskInput createInputInternal(HadoopTaskContext 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 HadoopTaskInput createInput(HadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param ctx Task info.
+     * @return Output.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected abstract HadoopTaskOutput createOutput(HadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param ctx Task info.
+     * @return Task output.
+     * @throws IgniteCheckedException If failed.
+     */
+    private HadoopTaskOutput createOutputInternal(HadoopTaskContext 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 HadoopTaskInfo taskInfo() {
+        return info;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..39b4935
--- /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 HadoopJob job, Collection<HadoopTaskInfo> 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(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via
+     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called.
+     *
+     * @param jobId Job ID to cancel.
+     */
+    public abstract void cancelTasks(HadoopJobId jobId) throws IgniteCheckedException;
+
+    /**
+     * On job state change callback;
+     *
+     * @param meta Job metadata.
+     */
+    public abstract void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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..c5ee16c
--- /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.counter.*;
+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 HadoopCounters 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 HadoopCounters 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 HadoopCounters 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 = (HadoopCounters)in.readObject();
+    }
+}


[34/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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/6423cf02/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).");
-    }
-}


[04/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..76988a3
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
@@ -0,0 +1,1006 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.hadoop.mapreduce.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.mapreduce.*;
+import org.apache.ignite.igfs.secondary.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.igfs.*;
+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 HadoopAbstractSelfTest {
+    /** */
+    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 IgniteFileSystem IGFS = new MockIgfs();
+
+    /** Planner. */
+    private static final HadoopMapReducePlanner PLANNER = new IgniteHadoopMapReducePlanner();
+
+    /** 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<HadoopMapReducePlan> 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 {
+        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1);
+        HadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_2);
+        HadoopFileBlock 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 {
+        HadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1);
+        HadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_2);
+        HadoopFileBlock 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 {
+        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2);
+        HadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2);
+        HadoopFileBlock 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 {
+        HadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1, HOST_2);
+        HadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_1, HOST_2);
+        HadoopFileBlock 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 {
+        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2, HOST_3);
+        HadoopFileBlock 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 {
+        HadoopFileBlock split1 = split(false, "/file1", 0, 100, INVALID_HOST_1, INVALID_HOST_2);
+        HadoopFileBlock split2 = split(false, "/file2", 0, 100, INVALID_HOST_1, INVALID_HOST_3);
+        HadoopFileBlock 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 HadoopMapReducePlan plan(int reducers, HadoopInputSplit... splits) throws IgniteCheckedException {
+        assert reducers > 0;
+        assert splits != null && splits.length > 0;
+
+        Collection<HadoopInputSplit> 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);
+
+        HadoopMapReducePlan 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, HadoopInputSplit... expSplits) {
+        Collection<HadoopInputSplit> expSplitsCol = new ArrayList<>();
+
+        Collections.addAll(expSplitsCol, expSplits);
+
+        Collection<HadoopInputSplit> 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 HadoopFileBlock split(boolean igfs, String file, long start, long len, String... hosts) {
+        URI uri = URI.create((igfs ? "igfs://igfs@" : "hdfs://") + file);
+
+        return new HadoopFileBlock(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 HadoopJob {
+        /** Reducers count. */
+        private final int reducers;
+
+        /** */
+        private Collection<HadoopInputSplit> splitList;
+
+        /**
+         * Constructor.
+         *
+         * @param reducers Reducers count.
+         * @param splitList Splits.
+         */
+        private MockJob(int reducers, Collection<HadoopInputSplit> splitList) {
+            this.reducers = reducers;
+            this.splitList = splitList;
+        }
+
+        /** {@inheritDoc} */
+        @Override public HadoopJobId id() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public HadoopJobInfo info() {
+            return new HadoopDefaultJobInfo() {
+                @Override public int reducers() {
+                    return reducers;
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<HadoopInputSplit> input() throws IgniteCheckedException {
+            return splitList;
+        }
+
+        /** {@inheritDoc} */
+        @Override public HadoopTaskContext getTaskContext(HadoopTaskInfo 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(HadoopTaskInfo info) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cleanupTaskEnvironment(HadoopTaskInfo 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 FileSystemConfiguration 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 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 IgniteFileSystem withAsync() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isAsync() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <R> IgniteFuture<R> future() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsSecondaryFileSystem asSecondary() {
+            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 IgniteFileSystem igfsx(String name) {
+            assert F.eq("igfs", name);
+
+            return IGFS;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Hadoop 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/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
new file mode 100644
index 0000000..8cf31a2
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
+import org.apache.ignite.testframework.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Test file systems for the working directory multi-threading support.
+ */
+public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
+    private static final int THREAD_COUNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+
+
+    /**
+     * Test the file system with specified URI for the multi-thread working directory support.
+     *
+     * @param uri Base URI of the file system (scheme and authority).
+     * @throws Exception If fails.
+     */
+    private void testFileSystem(final URI uri) throws Exception {
+        final Configuration cfg = new Configuration();
+
+        setupFileSystems(cfg);
+
+        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);
+        final CountDownLatch changeDirPhase = new CountDownLatch(THREAD_COUNT);
+        final CountDownLatch changeAbsDirPhase = new CountDownLatch(THREAD_COUNT);
+        final CountDownLatch finishPhase = new CountDownLatch(THREAD_COUNT);
+
+        final Path[] newUserInitWorkDir = new Path[THREAD_COUNT];
+        final Path[] newWorkDir = new Path[THREAD_COUNT];
+        final Path[] newAbsWorkDir = new Path[THREAD_COUNT];
+        final Path[] newInstanceWorkDir = new Path[THREAD_COUNT];
+
+        final AtomicInteger threadNum = new AtomicInteger(0);
+
+        GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    int curThreadNum = threadNum.getAndIncrement();
+
+                    FileSystem fs = FileSystem.get(uri, cfg);
+
+                    HadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
+
+                    if ("file".equals(uri.getScheme()))
+                        FileSystem.get(uri, cfg).setWorkingDirectory(new Path("file:///user/user" + curThreadNum));
+
+                    changeUserPhase.countDown();
+                    changeUserPhase.await();
+
+                    newUserInitWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
+
+                    FileSystem.get(uri, cfg).setWorkingDirectory(new Path("folder" + curThreadNum));
+
+                    changeDirPhase.countDown();
+                    changeDirPhase.await();
+
+                    newWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
+
+                    FileSystem.get(uri, cfg).setWorkingDirectory(new Path("/folder" + curThreadNum));
+
+                    changeAbsDirPhase.countDown();
+                    changeAbsDirPhase.await();
+
+                    newAbsWorkDir[curThreadNum] = FileSystem.get(uri, cfg).getWorkingDirectory();
+
+                    newInstanceWorkDir[curThreadNum] = FileSystem.newInstance(uri, cfg).getWorkingDirectory();
+
+                    finishPhase.countDown();
+                }
+                catch (InterruptedException | IOException e) {
+                    error("Failed to execute test thread.", e);
+
+                    fail();
+                }
+            }
+        }, THREAD_COUNT, "filesystems-test");
+
+        finishPhase.await();
+
+        for (int i = 0; i < THREAD_COUNT; i ++) {
+            cfg.set(MRJobConfig.USER_NAME, "user" + i);
+
+            Path workDir = new Path(new Path(uri), "user/user" + i);
+
+            cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, workDir.toString());
+
+            assertEquals(workDir, FileSystem.newInstance(uri, cfg).getWorkingDirectory());
+
+            assertEquals(workDir, newUserInitWorkDir[i]);
+
+            assertEquals(new Path(new Path(uri), "user/user" + i + "/folder" + i), newWorkDir[i]);
+
+            assertEquals(new Path("/folder" + i), newAbsWorkDir[i]);
+
+            assertEquals(new Path(new Path(uri), "user/" + System.getProperty("user.name")), newInstanceWorkDir[i]);
+        }
+
+        System.out.println(System.getProperty("user.dir"));
+    }
+
+    /**
+     * Test IGFS multi-thread working directory.
+     *
+     * @throws Exception If fails.
+     */
+    public void testIgfs() throws Exception {
+        testFileSystem(URI.create(igfsScheme()));
+    }
+
+    /**
+     * Test HDFS multi-thread working directory.
+     *
+     * @throws Exception If fails.
+     */
+    public void testHdfs() throws Exception {
+        testFileSystem(URI.create("hdfs://localhost/"));
+    }
+
+    /**
+     * Test LocalFS multi-thread working directory.
+     *
+     * @throws Exception If fails.
+     */
+    public void testLocal() throws Exception {
+        testFileSystem(URI.create("file:///"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java
new file mode 100644
index 0000000..e385ca7
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java
@@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mapreduce.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Grouping test.
+ */
+public class HadoopGroupingTest extends HadoopAbstractSelfTest {
+    /** */
+    private static final String PATH_OUTPUT = "/test-out";
+
+    /** */
+    private static final GridConcurrentHashSet<UUID> vals = HadoopSharedMap.map(HadoopGroupingTest.class)
+        .put("vals", new GridConcurrentHashSet<UUID>());
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    protected boolean igfsEnabled() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGroupingReducer() throws Exception {
+        doTestGrouping(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGroupingCombiner() throws Exception {
+        doTestGrouping(true);
+    }
+
+    /**
+     * @param combiner With combiner.
+     * @throws Exception If failed.
+     */
+    public void doTestGrouping(boolean combiner) throws Exception {
+        vals.clear();
+
+        Job job = Job.getInstance();
+
+        job.setInputFormatClass(InFormat.class);
+        job.setOutputFormatClass(OutFormat.class);
+
+        job.setOutputKeyClass(YearTemperature.class);
+        job.setOutputValueClass(Text.class);
+
+        job.setMapperClass(Mapper.class);
+
+        if (combiner) {
+            job.setCombinerClass(MyReducer.class);
+            job.setNumReduceTasks(0);
+            job.setCombinerKeyGroupingComparatorClass(YearComparator.class);
+        }
+        else {
+            job.setReducerClass(MyReducer.class);
+            job.setNumReduceTasks(4);
+            job.setGroupingComparatorClass(YearComparator.class);
+        }
+
+        grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 2),
+            createJobInfo(job.getConfiguration())).get(30000);
+
+        assertTrue(vals.isEmpty());
+    }
+
+    public static class MyReducer extends Reducer<YearTemperature, Text, Text, Object> {
+        /** */
+        int lastYear;
+
+        @Override protected void reduce(YearTemperature key, Iterable<Text> vals0, Context context)
+            throws IOException, InterruptedException {
+            X.println("___ : " + context.getTaskAttemptID() + " --> " + key);
+
+            Set<UUID> ids = new HashSet<>();
+
+            for (Text val : vals0)
+                assertTrue(ids.add(UUID.fromString(val.toString())));
+
+            for (Text val : vals0)
+                assertTrue(ids.remove(UUID.fromString(val.toString())));
+
+            assertTrue(ids.isEmpty());
+
+            assertTrue(key.year > lastYear);
+
+            lastYear = key.year;
+
+            for (Text val : vals0)
+                assertTrue(vals.remove(UUID.fromString(val.toString())));
+        }
+    }
+
+    public static class YearComparator implements RawComparator<YearTemperature> { // Grouping comparator.
+        /** {@inheritDoc} */
+        @Override public int compare(YearTemperature o1, YearTemperature o2) {
+            return Integer.compare(o1.year, o2.year);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+            throw new IllegalStateException();
+        }
+    }
+
+    public static class YearTemperature implements WritableComparable<YearTemperature>, Cloneable {
+        /** */
+        private int year;
+
+        /** */
+        private int temperature;
+
+        /** {@inheritDoc} */
+        @Override public void write(DataOutput out) throws IOException {
+            out.writeInt(year);
+            out.writeInt(temperature);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readFields(DataInput in) throws IOException {
+            year = in.readInt();
+            temperature = in.readInt();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            throw new IllegalStateException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() { // To be partitioned by year.
+            return year;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(YearTemperature o) {
+            int res = Integer.compare(year, o.year);
+
+            if (res != 0)
+                return res;
+
+            // Sort comparator by year and temperature, to find max for year.
+            return Integer.compare(o.temperature, temperature);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(YearTemperature.class, this);
+        }
+    }
+
+    public static class InFormat extends InputFormat<YearTemperature, Text> {
+        /** {@inheritDoc} */
+        @Override public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
+            ArrayList<InputSplit> list = new ArrayList<>();
+
+            for (int i = 0; i < 10; i++)
+                list.add(new HadoopSortingTest.FakeSplit(20));
+
+            return list;
+        }
+
+        /** {@inheritDoc} */
+        @Override public RecordReader<YearTemperature, Text> createRecordReader(final InputSplit split,
+            TaskAttemptContext context) throws IOException, InterruptedException {
+            return new RecordReader<YearTemperature, Text>() {
+                /** */
+                int cnt;
+
+                /** */
+                Random rnd = new GridRandom();
+
+                /** */
+                YearTemperature key = new YearTemperature();
+
+                /** */
+                Text val = new Text();
+
+                @Override public void initialize(InputSplit split, TaskAttemptContext context) {
+                    // No-op.
+                }
+
+                @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+                    return cnt++ < split.getLength();
+                }
+
+                @Override public YearTemperature getCurrentKey() {
+                    key.year = 1990 + rnd.nextInt(10);
+                    key.temperature = 10 + rnd.nextInt(20);
+
+                    return key;
+                }
+
+                @Override public Text getCurrentValue() {
+                    UUID id = UUID.randomUUID();
+
+                    assertTrue(vals.add(id));
+
+                    val.set(id.toString());
+
+                    return val;
+                }
+
+                @Override public float getProgress() {
+                    return 0;
+                }
+
+                @Override public void close() {
+                    // No-op.
+                }
+            };
+        }
+    }
+
+    /**
+     *
+     */
+    public static class OutFormat extends OutputFormat {
+        /** {@inheritDoc} */
+        @Override public RecordWriter getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java
new file mode 100644
index 0000000..943d89f
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Job tracker self test.
+ */
+public class HadoopJobTrackerSelfTest extends HadoopAbstractSelfTest {
+    /** */
+    private static final String PATH_OUTPUT = "/test-out";
+
+    /** Test block count parameter name. */
+    private static final int BLOCK_CNT = 10;
+
+    /** */
+    private static HadoopSharedMap m = HadoopSharedMap.map(HadoopJobTrackerSelfTest.class);
+
+    /** Map task execution count. */
+    private static final AtomicInteger mapExecCnt = m.put("mapExecCnt", new AtomicInteger());
+
+    /** Reduce task execution count. */
+    private static final AtomicInteger reduceExecCnt = m.put("reduceExecCnt", new AtomicInteger());
+
+    /** Reduce task execution count. */
+    private static final AtomicInteger combineExecCnt = m.put("combineExecCnt", new AtomicInteger());
+
+    /** */
+    private static final Map<String, CountDownLatch> latch = m.put("latch", new HashMap<String, CountDownLatch>());
+
+    /** {@inheritDoc} */
+    @Override protected boolean igfsEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        latch.put("mapAwaitLatch", new CountDownLatch(1));
+        latch.put("reduceAwaitLatch", new CountDownLatch(1));
+        latch.put("combineAwaitLatch", new CountDownLatch(1));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        mapExecCnt.set(0);
+        combineExecCnt.set(0);
+        reduceExecCnt.set(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setMapReducePlanner(new HadoopTestRoundRobinMrPlanner());
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleTaskSubmit() throws Exception {
+        try {
+            UUID globalId = UUID.randomUUID();
+
+            Job job = Job.getInstance();
+            setupFileSystems(job.getConfiguration());
+
+            job.setMapperClass(TestMapper.class);
+            job.setReducerClass(TestReducer.class);
+            job.setInputFormatClass(InFormat.class);
+
+            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT + "1"));
+
+            HadoopJobId jobId = new HadoopJobId(globalId, 1);
+
+            grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+
+            checkStatus(jobId, false);
+
+            info("Releasing map latch.");
+
+            latch.get("mapAwaitLatch").countDown();
+
+            checkStatus(jobId, false);
+
+            info("Releasing reduce latch.");
+
+            latch.get("reduceAwaitLatch").countDown();
+
+            checkStatus(jobId, true);
+
+            assertEquals(10, mapExecCnt.get());
+            assertEquals(0, combineExecCnt.get());
+            assertEquals(1, reduceExecCnt.get());
+        }
+        finally {
+            // Safety.
+            latch.get("mapAwaitLatch").countDown();
+            latch.get("combineAwaitLatch").countDown();
+            latch.get("reduceAwaitLatch").countDown();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTaskWithCombinerPerMap() throws Exception {
+        try {
+            UUID globalId = UUID.randomUUID();
+
+            Job job = Job.getInstance();
+            setupFileSystems(job.getConfiguration());
+
+            job.setMapperClass(TestMapper.class);
+            job.setReducerClass(TestReducer.class);
+            job.setCombinerClass(TestCombiner.class);
+            job.setInputFormatClass(InFormat.class);
+
+            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT + "2"));
+
+            HadoopJobId jobId = new HadoopJobId(globalId, 1);
+
+            grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+
+            checkStatus(jobId, false);
+
+            info("Releasing map latch.");
+
+            latch.get("mapAwaitLatch").countDown();
+
+            checkStatus(jobId, false);
+
+            // All maps are completed. We have a combiner, so no reducers should be executed
+            // before combiner latch is released.
+
+            U.sleep(50);
+
+            assertEquals(0, reduceExecCnt.get());
+
+            info("Releasing combiner latch.");
+
+            latch.get("combineAwaitLatch").countDown();
+
+            checkStatus(jobId, false);
+
+            info("Releasing reduce latch.");
+
+            latch.get("reduceAwaitLatch").countDown();
+
+            checkStatus(jobId, true);
+
+            assertEquals(10, mapExecCnt.get());
+            assertEquals(10, combineExecCnt.get());
+            assertEquals(1, reduceExecCnt.get());
+        }
+        finally {
+            // Safety.
+            latch.get("mapAwaitLatch").countDown();
+            latch.get("combineAwaitLatch").countDown();
+            latch.get("reduceAwaitLatch").countDown();
+        }
+    }
+
+    /**
+     * Checks job execution status.
+     *
+     * @param jobId Job ID.
+     * @param complete Completion status.
+     * @throws Exception If failed.
+     */
+    private void checkStatus(HadoopJobId jobId, boolean complete) throws Exception {
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteKernal kernal = (IgniteKernal)grid(i);
+
+            Hadoop hadoop = kernal.hadoop();
+
+            HadoopJobStatus stat = hadoop.status(jobId);
+
+            assert stat != null;
+
+            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
+
+            if (!complete)
+                assertFalse(fut.isDone());
+            else {
+                info("Waiting for status future completion on node [idx=" + i + ", nodeId=" +
+                    kernal.getLocalNodeId() + ']');
+
+                fut.get();
+            }
+        }
+    }
+
+    /**
+     * Test input format
+     */
+    public static class InFormat extends InputFormat {
+
+        @Override public List<InputSplit> getSplits(JobContext ctx) throws IOException, InterruptedException {
+            List<InputSplit> res = new ArrayList<>(BLOCK_CNT);
+
+            for (int i = 0; i < BLOCK_CNT; i++)
+                try {
+                    res.add(new FileSplit(new Path(new URI("someFile")), i, i + 1, new String[] {"localhost"}));
+                }
+                catch (URISyntaxException e) {
+                    throw new IOException(e);
+                }
+
+            return res;
+        }
+
+        @Override public RecordReader createRecordReader(InputSplit split, TaskAttemptContext ctx) throws IOException, InterruptedException {
+            return new RecordReader() {
+                @Override public void initialize(InputSplit split, TaskAttemptContext ctx) {
+                }
+
+                @Override public boolean nextKeyValue() {
+                    return false;
+                }
+
+                @Override public Object getCurrentKey() {
+                    return null;
+                }
+
+                @Override public Object getCurrentValue() {
+                    return null;
+                }
+
+                @Override public float getProgress() {
+                    return 0;
+                }
+
+                @Override public void close() {
+
+                }
+            };
+        }
+    }
+
+    /**
+     * Test mapper.
+     */
+    private static class TestMapper extends Mapper {
+        @Override public void run(Context ctx) throws IOException, InterruptedException {
+            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
+
+            latch.get("mapAwaitLatch").await();
+
+            mapExecCnt.incrementAndGet();
+
+            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
+        }
+    }
+
+    /**
+     * Test reducer.
+     */
+    private static class TestReducer extends Reducer {
+        @Override public void run(Context ctx) throws IOException, InterruptedException {
+            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
+
+            latch.get("reduceAwaitLatch").await();
+
+            reduceExecCnt.incrementAndGet();
+
+            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
+        }
+    }
+
+    /**
+     * Test combiner.
+     */
+    private static class TestCombiner extends Reducer {
+        @Override public void run(Context ctx) throws IOException, InterruptedException {
+            System.out.println("Running task: " + ctx.getTaskAttemptID().getTaskID().getId());
+
+            latch.get("combineAwaitLatch").await();
+
+            combineExecCnt.incrementAndGet();
+
+            System.out.println("Completed task: " + ctx.getTaskAttemptID().getTaskID().getId());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java
new file mode 100644
index 0000000..4a6e1ef
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.io.serializer.*;
+import org.apache.hadoop.mapred.*;
+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.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.examples.*;
+
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Tests map-reduce execution with embedded mode.
+ */
+public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest {
+    /** */
+    private static Map<String, Boolean> flags = HadoopSharedMap.map(HadoopMapReduceEmbeddedSelfTest.class)
+        .put("flags", new HashMap<String, Boolean>());
+
+    /** {@inheritDoc} */
+    @Override public HadoopConfiguration hadoopConfiguration(String gridName) {
+        HadoopConfiguration cfg = super.hadoopConfiguration(gridName);
+
+        cfg.setExternalExecution(false);
+
+        return cfg;
+    }
+
+    /**
+     * Tests whole job execution with all phases in old and new versions of API with definition of custom
+     * Serialization, Partitioner and IO formats.
+     * @throws Exception If fails.
+     */
+    public void testMultiReducerWholeMapReduceExecution() throws Exception {
+        IgfsPath inDir = new IgfsPath(PATH_INPUT);
+
+        igfs.mkdirs(inDir);
+
+        IgfsPath inFile = new IgfsPath(inDir, HadoopWordCount2.class.getSimpleName() + "-input");
+
+        generateTestFile(inFile.toString(), "key1", 10000, "key2", 20000, "key3", 15000, "key4", 7000, "key5", 12000,
+            "key6", 18000 );
+
+        for (int i = 0; i < 2; i++) {
+            boolean useNewAPI = i == 1;
+
+            igfs.delete(new IgfsPath(PATH_OUTPUT), true);
+
+            flags.put("serializationWasConfigured", false);
+            flags.put("partitionerWasConfigured", false);
+            flags.put("inputFormatWasConfigured", false);
+            flags.put("outputFormatWasConfigured", false);
+
+            JobConf jobConf = new JobConf();
+
+            jobConf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
+
+            //To split into about 6-7 items for v2
+            jobConf.setInt(FileInputFormat.SPLIT_MAXSIZE, 65000);
+
+            //For v1
+            jobConf.setInt("fs.local.block.size", 65000);
+
+            // File system coordinates.
+            setupFileSystems(jobConf);
+
+            HadoopWordCount1.setTasksClasses(jobConf, !useNewAPI, !useNewAPI, !useNewAPI);
+
+            if (!useNewAPI) {
+                jobConf.setPartitionerClass(CustomV1Partitioner.class);
+                jobConf.setInputFormat(CustomV1InputFormat.class);
+                jobConf.setOutputFormat(CustomV1OutputFormat.class);
+            }
+
+            Job job = Job.getInstance(jobConf);
+
+            HadoopWordCount2.setTasksClasses(job, useNewAPI, useNewAPI, useNewAPI);
+
+            if (useNewAPI) {
+                job.setPartitionerClass(CustomV2Partitioner.class);
+                job.setInputFormatClass(CustomV2InputFormat.class);
+                job.setOutputFormatClass(CustomV2OutputFormat.class);
+            }
+
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(IntWritable.class);
+
+            FileInputFormat.setInputPaths(job, new Path(igfsScheme() + inFile.toString()));
+            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
+
+            job.setNumReduceTasks(3);
+
+            job.setJarByClass(HadoopWordCount2.class);
+
+            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 1),
+                    createJobInfo(job.getConfiguration()));
+
+            fut.get();
+
+            assertTrue("Serialization was configured (new API is " + useNewAPI + ")",
+                 flags.get("serializationWasConfigured"));
+
+            assertTrue("Partitioner was configured (new API is = " + useNewAPI + ")",
+                 flags.get("partitionerWasConfigured"));
+
+            assertTrue("Input format was configured (new API is = " + useNewAPI + ")",
+                 flags.get("inputFormatWasConfigured"));
+
+            assertTrue("Output format was configured (new API is = " + useNewAPI + ")",
+                 flags.get("outputFormatWasConfigured"));
+
+            assertEquals("Use new API = " + useNewAPI,
+                "key3\t15000\n" +
+                "key6\t18000\n",
+                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00000")
+            );
+
+            assertEquals("Use new API = " + useNewAPI,
+                "key1\t10000\n" +
+                "key4\t7000\n",
+                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00001")
+            );
+
+            assertEquals("Use new API = " + useNewAPI,
+                "key2\t20000\n" +
+                "key5\t12000\n",
+                readAndSortFile(PATH_OUTPUT + "/" + (useNewAPI ? "part-r-" : "part-") + "00002")
+            );
+
+        }
+    }
+
+    /**
+     * Custom serialization class that inherits behaviour of native {@link WritableSerialization}.
+     */
+    protected static class CustomSerialization extends WritableSerialization {
+        @Override public void setConf(Configuration conf) {
+            super.setConf(conf);
+
+            flags.put("serializationWasConfigured", true);
+        }
+    }
+
+    /**
+     * Custom implementation of Partitioner in v1 API.
+     */
+    private static class CustomV1Partitioner extends org.apache.hadoop.mapred.lib.HashPartitioner {
+        /** {@inheritDoc} */
+        @Override public void configure(JobConf job) {
+            flags.put("partitionerWasConfigured", true);
+        }
+    }
+
+    /**
+     * Custom implementation of Partitioner in v2 API.
+     */
+    private static class CustomV2Partitioner extends org.apache.hadoop.mapreduce.lib.partition.HashPartitioner
+            implements Configurable {
+        /** {@inheritDoc} */
+        @Override public void setConf(Configuration conf) {
+            flags.put("partitionerWasConfigured", true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Configuration getConf() {
+            return null;
+        }
+    }
+
+    /**
+     * Custom implementation of InputFormat in v2 API.
+     */
+    private static class CustomV2InputFormat extends org.apache.hadoop.mapreduce.lib.input.TextInputFormat implements Configurable {
+        /** {@inheritDoc} */
+        @Override public void setConf(Configuration conf) {
+            flags.put("inputFormatWasConfigured", true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Configuration getConf() {
+            return null;
+        }
+    }
+
+    /**
+     * Custom implementation of OutputFormat in v2 API.
+     */
+    private static class CustomV2OutputFormat extends org.apache.hadoop.mapreduce.lib.output.TextOutputFormat implements Configurable {
+        /** {@inheritDoc} */
+        @Override public void setConf(Configuration conf) {
+            flags.put("outputFormatWasConfigured", true);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Configuration getConf() {
+            return null;
+        }
+    }
+
+    /**
+     * Custom implementation of InputFormat in v1 API.
+     */
+    private static class CustomV1InputFormat extends org.apache.hadoop.mapred.TextInputFormat {
+        /** {@inheritDoc} */
+        @Override public void configure(JobConf job) {
+            super.configure(job);
+
+            flags.put("inputFormatWasConfigured", true);
+        }
+    }
+
+    /**
+     * Custom implementation of OutputFormat in v1 API.
+     */
+    private static class CustomV1OutputFormat extends org.apache.hadoop.mapred.TextOutputFormat implements JobConfigurable {
+        /** {@inheritDoc} */
+        @Override public void configure(JobConf job) {
+            flags.put("outputFormatWasConfigured", true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
new file mode 100644
index 0000000..6242ecc
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.mapred.*;
+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.*;
+import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Test of whole cycle of map-reduce processing via Job tracker.
+ */
+public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /**
+     * Tests whole job execution with all phases in all combination of new and old versions of API.
+     * @throws Exception If fails.
+     */
+    public void testWholeMapReduceExecution() throws Exception {
+        IgfsPath inDir = new IgfsPath(PATH_INPUT);
+
+        igfs.mkdirs(inDir);
+
+        IgfsPath inFile = new IgfsPath(inDir, HadoopWordCount2.class.getSimpleName() + "-input");
+
+        generateTestFile(inFile.toString(), "red", 100000, "blue", 200000, "green", 150000, "yellow", 70000 );
+
+        for (int i = 0; i < 8; i++) {
+            igfs.delete(new IgfsPath(PATH_OUTPUT), true);
+
+            boolean useNewMapper = (i & 1) == 0;
+            boolean useNewCombiner = (i & 2) == 0;
+            boolean useNewReducer = (i & 4) == 0;
+
+            JobConf jobConf = new JobConf();
+
+            jobConf.set(JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName());
+            jobConf.setUser("yyy");
+            jobConf.set(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
+
+            //To split into about 40 items for v2
+            jobConf.setInt(FileInputFormat.SPLIT_MAXSIZE, 65000);
+
+            //For v1
+            jobConf.setInt("fs.local.block.size", 65000);
+
+            // File system coordinates.
+            setupFileSystems(jobConf);
+
+            HadoopWordCount1.setTasksClasses(jobConf, !useNewMapper, !useNewCombiner, !useNewReducer);
+
+            Job job = Job.getInstance(jobConf);
+
+            HadoopWordCount2.setTasksClasses(job, useNewMapper, useNewCombiner, useNewReducer);
+
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(IntWritable.class);
+
+            FileInputFormat.setInputPaths(job, new Path(igfsScheme() + inFile.toString()));
+            FileOutputFormat.setOutputPath(job, new Path(igfsScheme() + PATH_OUTPUT));
+
+            job.setJarByClass(HadoopWordCount2.class);
+
+            HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 1);
+
+            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+
+            fut.get();
+
+            checkJobStatistics(jobId);
+
+            assertEquals("Use new mapper: " + useNewMapper + ", new combiner: " + useNewCombiner + ", new reducer: " +
+                useNewReducer,
+                "blue\t200000\n" +
+                "green\t150000\n" +
+                "red\t100000\n" +
+                "yellow\t70000\n",
+                readAndSortFile(PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000")
+            );
+        }
+    }
+
+    /**
+     * Simple test job statistics.
+     *
+     * @param jobId Job id.
+     * @throws IgniteCheckedException
+     */
+    private void checkJobStatistics(HadoopJobId jobId) throws IgniteCheckedException, IOException {
+        HadoopCounters cntrs = grid(0).hadoop().counters(jobId);
+
+        HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null);
+
+        Map<String, SortedMap<Integer,Long>> tasks = new TreeMap<>();
+
+        Map<String, Integer> phaseOrders = new HashMap<>();
+        phaseOrders.put("submit", 0);
+        phaseOrders.put("prepare", 1);
+        phaseOrders.put("start", 2);
+        phaseOrders.put("Cstart", 3);
+        phaseOrders.put("finish", 4);
+
+        String prevTaskId = null;
+
+        long apiEvtCnt = 0;
+
+        for (T2<String, Long> evt : perfCntr.evts()) {
+            //We expect string pattern: COMBINE 1 run 7fa86a14-5a08-40e3-a7cb-98109b52a706
+            String[] parsedEvt = evt.get1().split(" ");
+
+            String taskId;
+            String taskPhase;
+
+            if ("JOB".equals(parsedEvt[0])) {
+                taskId = parsedEvt[0];
+                taskPhase = parsedEvt[1];
+            }
+            else {
+                taskId = ("COMBINE".equals(parsedEvt[0]) ? "MAP" : parsedEvt[0].substring(0, 3)) + parsedEvt[1];
+                taskPhase = ("COMBINE".equals(parsedEvt[0]) ? "C" : "") + parsedEvt[2];
+            }
+
+            if (!taskId.equals(prevTaskId))
+                tasks.put(taskId, new TreeMap<Integer,Long>());
+
+            Integer pos = phaseOrders.get(taskPhase);
+
+            assertNotNull("Invalid phase " + taskPhase, pos);
+
+            tasks.get(taskId).put(pos, evt.get2());
+
+            prevTaskId = taskId;
+
+            apiEvtCnt++;
+        }
+
+        for (Map.Entry<String ,SortedMap<Integer,Long>> task : tasks.entrySet()) {
+            Map<Integer, Long> order = task.getValue();
+
+            long prev = 0;
+
+            for (Map.Entry<Integer, Long> phase : order.entrySet()) {
+                assertTrue("Phase order of " + task.getKey() + " is invalid", phase.getValue() >= prev);
+
+                prev = phase.getValue();
+            }
+        }
+
+        final IgfsPath statPath = new IgfsPath("/xxx/yyy/zzz/" + jobId + "/performance");
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return igfs.exists(statPath);
+            }
+        }, 10000);
+
+        BufferedReader reader = new BufferedReader(new InputStreamReader(igfs.open(statPath)));
+
+        assertEquals(apiEvtCnt, HadoopTestUtils.simpleCheckJobStatFile(reader));
+    }
+}


[27/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..39f42b2
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -0,0 +1,1626 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.counter.HadoopCounters;
+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.HadoopJobPhase.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.*;
+import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.*;
+
+/**
+ * Hadoop job tracker.
+ */
+public class HadoopJobTracker extends HadoopComponent {
+    /** */
+    private final GridMutex mux = new GridMutex();
+
+    /** */
+    private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> jobMetaPrj;
+
+    /** Projection with expiry policy for finished job updates. */
+    private volatile GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> finishedJobMetaPrj;
+
+    /** Map-reduce execution planner. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private HadoopMapReducePlanner mrPlanner;
+
+    /** All the known jobs. */
+    private final ConcurrentMap<HadoopJobId, GridFutureAdapterEx<HadoopJob>> jobs = new ConcurrentHashMap8<>();
+
+    /** Locally active jobs. */
+    private final ConcurrentMap<HadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
+
+    /** Locally requested finish futures. */
+    private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJobId>> 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<HadoopJobId, HadoopJobMetadata> jobMetaCache() {
+        GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> 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<HadoopJobId, HadoopJobMetadata>)
+                        sysCache.projection(HadoopJobId.class, HadoopJobMetadata.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<HadoopJobId, HadoopJobMetadata> finishedJobMetaCache() {
+        GridCacheProjectionEx<HadoopJobId, HadoopJobMetadata> 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<HadoopJobId, HadoopJobMetadata>() {
+                @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends HadoopJobId,
+                    ? extends HadoopJobMetadata>> 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<HadoopJobId> 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<HadoopJobId> submit(HadoopJobId jobId, HadoopJobInfo 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);
+
+            HadoopJob job = job(jobId, info);
+
+            HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
+
+            HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info);
+
+            meta.mapReducePlan(mrPlan);
+
+            meta.pendingSplits(allSplits(mrPlan));
+            meta.pendingReducers(allReducers(mrPlan));
+
+            GridFutureAdapter<HadoopJobId> completeFut = new GridFutureAdapter<>();
+
+            GridFutureAdapter<HadoopJobId> 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();
+
+            HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.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 HadoopJobStatus status(HadoopJobMetadata meta) {
+        HadoopJobInfo jobInfo = meta.jobInfo();
+
+        return new HadoopJobStatus(
+            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 HadoopJobStatus status(HadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null; // Grid is stopping.
+
+        try {
+            HadoopJobMetadata 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(HadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null; // Grid is stopping.
+
+        try {
+            HadoopJobMetadata 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<HadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId,
+                new GridFutureAdapter<HadoopJobId>());
+
+            // 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 HadoopMapReducePlan plan(HadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null;
+
+        try {
+            HadoopJobMetadata 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(HadoopTaskInfo info, HadoopTaskStatus 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<HadoopJobId, HadoopJobMetadata> 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(HadoopJobId jobId, EntryProcessor<HadoopJobId, HadoopJobMetadata, 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(HadoopJobId jobId, Collection<Integer> reducers,
+        HadoopProcessDescriptor 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<HadoopInputSplit, Integer> allSplits(HadoopMapReducePlan plan) {
+        Map<HadoopInputSplit, Integer> res = new HashMap<>();
+
+        int taskNum = 0;
+
+        for (UUID nodeId : plan.mapperNodeIds()) {
+            for (HadoopInputSplit 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(HadoopMapReducePlan 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()) {
+                HadoopJobMetadata meta = (HadoopJobMetadata)metaObj;
+
+                HadoopJobId jobId = meta.jobId();
+
+                HadoopMapReducePlan plan = meta.mapReducePlan();
+
+                HadoopJobPhase phase = meta.phase();
+
+                try {
+                    if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
+                        // Failover setup task.
+                        HadoopJob job = job(jobId, meta.jobInfo());
+
+                        Collection<HadoopTaskInfo> 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<HadoopInputSplit> cancelSplits = null;
+
+                        for (UUID nodeId : plan.mapperNodeIds()) {
+                            if (ctx.kernalContext().discovery().node(nodeId) == null) {
+                                // Node has left the grid.
+                                Collection<HadoopInputSplit> 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 HadoopJobId, ? extends HadoopJobMetadata>> updated)
+        throws IgniteCheckedException {
+        UUID locNodeId = ctx.localNodeId();
+
+        for (CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata> entry : updated) {
+            HadoopJobId jobId = entry.getKey();
+            HadoopJobMetadata 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(HadoopJobId jobId, HadoopMapReducePlan 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(HadoopJobId jobId, HadoopJobMetadata meta, UUID locNodeId)
+        throws IgniteCheckedException {
+        JobLocalState state = activeJobs.get(jobId);
+
+        HadoopJob job = job(jobId, meta.jobInfo());
+
+        HadoopMapReducePlan plan = meta.mapReducePlan();
+
+        switch (meta.phase()) {
+            case PHASE_SETUP: {
+                if (ctx.jobUpdateLeader()) {
+                    Collection<HadoopTaskInfo> 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<HadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta);
+
+                if (tasks != null)
+                    ctx.taskExecutor().run(job, tasks);
+
+                break;
+            }
+
+            case PHASE_REDUCE: {
+                if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) {
+                    HadoopTaskInfo info = new HadoopTaskInfo(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<HadoopTaskInfo> 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()) {
+                            HadoopTaskInfo info = new HadoopTaskInfo(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<HadoopInputSplit> cancelMappers = new ArrayList<>();
+                    Collection<Integer> cancelReducers = new ArrayList<>();
+
+                    Collection<HadoopInputSplit> mappers = plan.mappers(ctx.localNodeId());
+
+                    if (mappers != null) {
+                        for (HadoopInputSplit 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<HadoopJobId> 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);
+
+                            HadoopCounterWriter writer = (HadoopCounterWriter)cls.newInstance();
+
+                            HadoopCounters 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<HadoopTaskInfo> setupTask(HadoopJobId jobId) {
+        if (activeJobs.containsKey(jobId))
+            return null;
+        else {
+            initState(jobId);
+
+            return Collections.singleton(new HadoopTaskInfo(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<HadoopTaskInfo> mapperTasks(Iterable<HadoopInputSplit> mappers, HadoopJobMetadata meta) {
+        UUID locNodeId = ctx.localNodeId();
+        HadoopJobId jobId = meta.jobId();
+
+        JobLocalState state = activeJobs.get(jobId);
+
+        Collection<HadoopTaskInfo> tasks = null;
+
+        if (mappers != null) {
+            if (state == null)
+                state = initState(jobId);
+
+            for (HadoopInputSplit split : mappers) {
+                if (state.addMapper(split)) {
+                    if (log.isDebugEnabled())
+                        log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId +
+                            ", split=" + split + ']');
+
+                    HadoopTaskInfo taskInfo = new HadoopTaskInfo(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<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJob job) {
+        UUID locNodeId = ctx.localNodeId();
+        HadoopJobId jobId = job.id();
+
+        JobLocalState state = activeJobs.get(jobId);
+
+        Collection<HadoopTaskInfo> 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 + ']');
+
+                    HadoopTaskInfo taskInfo = new HadoopTaskInfo(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(HadoopJobId 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 HadoopJob job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException {
+        GridFutureAdapterEx<HadoopJob> fut = jobs.get(jobId);
+
+        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapterEx<HadoopJob>())) != null)
+            return fut.get();
+
+        fut = jobs.get(jobId);
+
+        HadoopJob job = null;
+
+        try {
+            if (jobInfo == null) {
+                HadoopJobMetadata 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(HadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return false; // Grid is stopping.
+
+        try {
+            HadoopJobMetadata 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 HadoopCounters jobCounters(HadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null;
+
+        try {
+            final HadoopJobMetadata 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<HadoopInputSplit> 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(HadoopInputSplit 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(HadoopInputSplit 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 HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
+            final HadoopJobId 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 HadoopTaskInfo taskInfo, HadoopTaskStatus status,
+            final StackedProcessor prev) {
+            final HadoopJobId 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(HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
+            HadoopJobId 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(HadoopTaskInfo taskInfo, HadoopTaskStatus status,
+            final StackedProcessor prev) {
+            final HadoopJobId 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 HadoopJobPhase phase;
+
+        /**
+         * @param prev Previous closure.
+         * @param phase Phase to update.
+         */
+        private UpdatePhaseProcessor(@Nullable StackedProcessor prev, HadoopJobPhase phase) {
+            super(prev);
+
+            this.phase = phase;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata 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<HadoopInputSplit> splits;
+
+        /** Error. */
+        private final Throwable err;
+
+        /**
+         * @param prev Previous closure.
+         * @param split Mapper split to remove.
+         * @param err Error.
+         */
+        private RemoveMappersProcessor(@Nullable StackedProcessor prev, HadoopInputSplit 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<HadoopInputSplit> splits,
+            Throwable err) {
+            super(prev);
+
+            this.splits = splits;
+            this.err = err;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+            Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+
+            for (HadoopInputSplit 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(HadoopJobMetadata meta, HadoopJobMetadata 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 HadoopProcessDescriptor desc;
+
+        /**
+         * @param prev Previous closure.
+         * @param rdc Reducers to initialize.
+         * @param desc External process descriptor.
+         */
+        private InitializeReducersProcessor(@Nullable StackedProcessor prev,
+            Collection<Integer> rdc,
+            HadoopProcessDescriptor desc) {
+            super(prev);
+
+            assert !F.isEmpty(rdc);
+            assert desc != null;
+
+            this.rdc = rdc;
+            this.desc = desc;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+            Map<Integer, HadoopProcessDescriptor> oldMap = meta.reducersAddresses();
+
+            Map<Integer, HadoopProcessDescriptor> rdcMap = oldMap == null ?
+                new HashMap<Integer, HadoopProcessDescriptor>() : 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<HadoopInputSplit> 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<HadoopInputSplit> 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<HadoopInputSplit> splits,
+            Collection<Integer> rdc) {
+            super(prev);
+
+            this.splits = splits;
+            this.rdc = rdc;
+            this.err = err;
+        }
+
+        /** {@inheritDoc} */
+        @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());
+
+            if (rdc != null)
+                rdcCp.removeAll(rdc);
+
+            cp.pendingReducers(rdcCp);
+
+            Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+
+            if (splits != null) {
+                for (HadoopInputSplit 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 HadoopCounters counters;
+
+        /**
+         * @param prev Previous closure.
+         * @param counters Task counters to add into job counters.
+         */
+        private IncrementCountersProcessor(@Nullable StackedProcessor prev, HadoopCounters counters) {
+            super(prev);
+
+            assert counters != null;
+
+            this.counters = counters;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+            HadoopCounters cntrs = new HadoopCountersImpl(cp.counters());
+
+            cntrs.merge(counters);
+
+            cp.counters(cntrs);
+        }
+    }
+
+    /**
+     * Abstract stacked closure.
+     */
+    private abstract static class StackedProcessor implements
+        EntryProcessor<HadoopJobId, HadoopJobMetadata, 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<HadoopJobId, HadoopJobMetadata> e, Object... args) {
+            HadoopJobMetadata val = apply(e.getValue());
+
+            if (val != null)
+                e.setValue(val);
+            else
+                e.remove();;
+
+            return null;
+        }
+
+        /**
+         * @param meta Old value.
+         * @return New value.
+         */
+        private HadoopJobMetadata apply(HadoopJobMetadata meta) {
+            if (meta == null)
+                return null;
+
+            HadoopJobMetadata cp = prev != null ? prev.apply(meta) : new HadoopJobMetadata(meta);
+
+            update(meta, cp);
+
+            return cp;
+        }
+
+        /**
+         * Update given job metadata object.
+         *
+         * @param meta Initial job metadata.
+         * @param cp Copy.
+         */
+        protected abstract void update(HadoopJobMetadata meta, HadoopJobMetadata cp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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/6423cf02/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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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/6423cf02/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..1413612
--- /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 HadoopMapReducePlan {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Mappers map. */
+    private Map<UUID, Collection<HadoopInputSplit>> 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<HadoopInputSplit>> mappers,
+        Map<UUID, int[]> reducers) {
+        this.mappers = mappers;
+        this.reducers = reducers;
+
+        if (mappers != null) {
+            for (Collection<HadoopInputSplit> 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<HadoopInputSplit> 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/6423cf02/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/6423cf02/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);
-    }
-}


[28/58] [abbrv] incubator-ignite git commit: IGNITE-386: Squashed changes.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/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..3f574e9
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.counter.HadoopCounters;
+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.HadoopJobPhase.*;
+
+/**
+ * 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 HadoopJobId jobId;
+
+    /** Job info. */
+    private HadoopJobInfo jobInfo;
+
+    /** Node submitted job. */
+    private UUID submitNodeId;
+
+    /** Map-reduce plan. */
+    private HadoopMapReducePlan mrPlan;
+
+    /** Pending splits for which mapper should be executed. */
+    private Map<HadoopInputSplit, Integer> pendingSplits;
+
+    /** Pending reducers. */
+    private Collection<Integer> pendingReducers;
+
+    /** Reducers addresses. */
+    @GridToStringInclude
+    private Map<Integer, HadoopProcessDescriptor> reducersAddrs;
+
+    /** Job phase. */
+    private HadoopJobPhase phase = PHASE_SETUP;
+
+    /** Fail cause. */
+    @GridToStringExclude
+    private Throwable failCause;
+
+    /** Version. */
+    private long ver;
+
+    /** Job counters */
+    private HadoopCounters 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, HadoopJobId jobId, HadoopJobInfo 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(HadoopJobPhase phase) {
+        this.phase = phase;
+    }
+
+    /**
+     * @return Job phase.
+     */
+    public HadoopJobPhase 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<HadoopInputSplit, Integer> pendingSplits) {
+        this.pendingSplits = pendingSplits;
+    }
+
+    /**
+     * Gets collection of pending splits.
+     *
+     * @return Collection of pending splits.
+     */
+    public Map<HadoopInputSplit, 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 HadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @param mrPlan Map-reduce plan.
+     */
+    public void mapReducePlan(HadoopMapReducePlan mrPlan) {
+        assert this.mrPlan == null : "Map-reduce plan can only be initialized once.";
+
+        this.mrPlan = mrPlan;
+    }
+
+    /**
+     * @return Map-reduce plan.
+     */
+    public HadoopMapReducePlan mapReducePlan() {
+        return mrPlan;
+    }
+
+    /**
+     * @return Job info.
+     */
+    public HadoopJobInfo jobInfo() {
+        return jobInfo;
+    }
+
+    /**
+     * Returns job counters.
+     *
+     * @return Collection of counters.
+     */
+    public HadoopCounters counters() {
+        return counters;
+    }
+
+    /**
+     * Sets counters.
+     *
+     * @param counters Collection of counters.
+     */
+    public void counters(HadoopCounters 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(HadoopInputSplit 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 = (HadoopJobId)in.readObject();
+        jobInfo = (HadoopJobInfo)in.readObject();
+        mrPlan = (HadoopMapReducePlan)in.readObject();
+        pendingSplits = (Map<HadoopInputSplit,Integer>)in.readObject();
+        pendingReducers = (Collection<Integer>)in.readObject();
+        phase = (HadoopJobPhase)in.readObject();
+        failCause = (Throwable)in.readObject();
+        ver = in.readLong();
+        reducersAddrs = (Map<Integer, HadoopProcessDescriptor>)in.readObject();
+        counters = (HadoopCounters)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());
+    }
+}


[56/58] [abbrv] incubator-ignite git commit: # IGNITE-371: Applied Ivan V. patch for fixing "usedSpace" secondary file system issue.

Posted by yz...@apache.org.
# IGNITE-371: Applied Ivan V. patch for fixing "usedSpace" secondary file system issue.


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

Branch: refs/heads/ignite-313
Commit: ae413fb5871436f7d57761d0dca85a5aa68b2a37
Parents: 49af1cf
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Mar 5 11:57:43 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Mar 5 11:57:43 2015 +0300

----------------------------------------------------------------------
 .../ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae413fb5/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index 98f2e46..f5d7055 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -390,7 +390,9 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public long usedSpaceSize() {
         try {
-            return fileSys.getContentSummary(new Path(fileSys.getUri())).getSpaceConsumed();
+            // We don't use FileSystem#getUsed() since it counts only the files
+            // in the filesystem root, not all the files recursively.
+            return fileSys.getContentSummary(new Path("/")).getSpaceConsumed();
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to get used space size of file system.");