You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/02/03 13:28:11 UTC

[16/28] ignite git commit: ignite-2080 Data alignment issues with Unsafe

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/internal/util/io/GridUnsafeDataInputOutputByteOrderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/io/GridUnsafeDataInputOutputByteOrderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/io/GridUnsafeDataInputOutputByteOrderSelfTest.java
new file mode 100644
index 0000000..f3ff781
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/io/GridUnsafeDataInputOutputByteOrderSelfTest.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.io;
+
+import java.io.ByteArrayInputStream;
+import java.util.Random;
+import junit.framework.TestCase;
+
+import static org.apache.ignite.GridTestIoUtils.getCharByByteLE;
+import static org.apache.ignite.GridTestIoUtils.getDoubleByByteLE;
+import static org.apache.ignite.GridTestIoUtils.getFloatByByteLE;
+import static org.apache.ignite.GridTestIoUtils.getIntByByteLE;
+import static org.apache.ignite.GridTestIoUtils.getLongByByteLE;
+import static org.apache.ignite.GridTestIoUtils.getShortByByteLE;
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Grid unsafe data input/output byte order sanity tests.
+ */
+public class GridUnsafeDataInputOutputByteOrderSelfTest extends TestCase {
+    /** Array length. */
+    private static final int ARR_LEN = 16;
+
+    /** Length bytes. */
+    private static final int LEN_BYTES = 4;
+
+    /** Rnd. */
+    private static Random RND = new Random();
+
+    /** Out. */
+    private GridUnsafeDataOutput out;
+
+    /** In. */
+    private GridUnsafeDataInput in;
+
+    /** {@inheritDoc} */
+    @Override protected void setUp() throws Exception {
+        out = new GridUnsafeDataOutput(16 * 8+ LEN_BYTES);
+        in = new GridUnsafeDataInput();
+        in.inputStream(new ByteArrayInputStream(out.internalArray()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        in.close();
+        out.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        short val = (short)RND.nextLong();
+
+        out.writeShort(val);
+
+        assertEquals(val, getShortByByteLE(out.internalArray()));
+        assertEquals(val, in.readShort());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortArray() throws Exception {
+        short[] arr = new short[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = (short)RND.nextLong();
+
+        out.writeShortArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getShortByByteLE(outArr, i * 2+ LEN_BYTES));
+
+        assertArrayEquals(arr, in.readShortArray());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testChar() throws Exception {
+        char val = (char)RND.nextLong();
+
+        out.writeChar(val);
+
+        assertEquals(val, getCharByByteLE(out.internalArray()));
+        assertEquals(val, in.readChar());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharArray() throws Exception {
+        char[] arr = new char[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = (char)RND.nextLong();
+
+        out.writeCharArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getCharByByteLE(outArr, i * 2+ LEN_BYTES));
+
+        assertArrayEquals(arr, in.readCharArray());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInt() throws Exception {
+        int val = RND.nextInt();
+
+        out.writeInt(val);
+
+        assertEquals(val, getIntByByteLE(out.internalArray()));
+        assertEquals(val, in.readInt());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntArray() throws Exception {
+        int[] arr = new int[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = RND.nextInt();
+
+        out.writeIntArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getIntByByteLE(outArr, i * 4+ LEN_BYTES));
+
+        assertArrayEquals(arr, in.readIntArray());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        long val = RND.nextLong();
+
+        out.writeLong(val);
+
+        assertEquals(val, getLongByByteLE(out.internalArray()));
+        assertEquals(val, in.readLong());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongArray() throws Exception {
+        long[] arr = new long[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = RND.nextLong();
+
+        out.writeLongArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getLongByByteLE(outArr, i * 8+ LEN_BYTES));
+
+        assertArrayEquals(arr, in.readLongArray());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        float val = RND.nextFloat();
+
+        out.writeFloat(val);
+
+        assertEquals(val, getFloatByByteLE(out.internalArray()), 0);
+        assertEquals(val, in.readFloat(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatArray() throws Exception {
+        float[] arr = new float[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = RND.nextFloat();
+
+        out.writeFloatArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getFloatByByteLE(outArr, i * 4+ LEN_BYTES), 0);
+
+        assertArrayEquals(arr, in.readFloatArray(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        double val = RND.nextDouble();
+
+        out.writeDouble(val);
+
+        assertEquals(val, getDoubleByByteLE(out.internalArray()), 0);
+        assertEquals(val, in.readDouble(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleArray() throws Exception {
+        double[] arr = new double[ARR_LEN];
+
+        for (int i = 0; i < ARR_LEN; i++)
+            arr[i] = RND.nextDouble();
+
+        out.writeDoubleArray(arr);
+
+        byte[] outArr = out.internalArray();
+
+        for (int i = 0; i < ARR_LEN; i++)
+            assertEquals(arr[i], getDoubleByByteLE(outArr, i * 8+ LEN_BYTES), 0);
+
+        assertArrayEquals(arr, in.readDoubleArray(), 0);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
index 37e7afe..1bbae51 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
@@ -50,7 +50,6 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentLinkedDeque8;
 import org.jsr166.ThreadLocalRandom8;
-import sun.misc.Unsafe;
 
 /**
  * Tests synchronization performance vs. lock.
@@ -948,16 +947,14 @@ public class GridBasicPerformanceTest {
 
         GridTimer t = new GridTimer("unsafe");
 
-        Unsafe unsafe = GridUnsafe.unsafe();
-
         int mem = 1024;
 
         for (int i = 0; i < MAX; i++) {
-            addrs[i] = unsafe.allocateMemory(mem);
+            addrs[i] = GridUnsafe.allocateMemory(mem);
 
-            unsafe.putByte(addrs[i] + RAND.nextInt(mem), (byte)RAND.nextInt(mem));
+            GridUnsafe.putByte(addrs[i] + RAND.nextInt(mem), (byte)RAND.nextInt(mem));
 
-            v = unsafe.getByte(addrs[i] + RAND.nextInt(mem));
+            v = GridUnsafe.getByte(addrs[i] + RAND.nextInt(mem));
         }
 
         X.println("Unsafe [time=" + t.stop() + "ms, v=" + v + ']');
@@ -965,7 +962,7 @@ public class GridBasicPerformanceTest {
         Thread.sleep(5000L);
 
         for (long l : addrs)
-            unsafe.freeMemory(l);
+            GridUnsafe.freeMemory(l);
     }
 
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 2d5b2c5..7bb2cf3 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -1101,7 +1101,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
         // On Windows and Mac machines two nodes can reside on the same port
         // (if one node has localHost="127.0.0.1" and another has localHost="0.0.0.0").
         // So two nodes do not even discover each other.
-        if (U.isWindows() || U.isMacOs())
+        if (U.isWindows() || U.isMacOs() || U.isSolaris())
             return;
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 27511ff..3c83d86 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -206,7 +206,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected <T> T allocateInstance(Class<T> cls) throws Exception {
-        return (T)GridUnsafe.unsafe().allocateInstance(cls);
+        return (T)GridUnsafe.allocateInstance(cls);
     }
 
     /**
@@ -215,7 +215,7 @@ public abstract class GridAbstractTest extends TestCase {
      */
     @Nullable protected <T> T allocateInstance0(Class<T> cls) {
         try {
-            return (T)GridUnsafe.unsafe().allocateInstance(cls);
+            return (T)GridUnsafe.allocateInstance(cls);
         }
         catch (InstantiationException e) {
             e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
index 5eb7b66..3ec55d0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java
@@ -46,6 +46,8 @@ import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompac
 import org.apache.ignite.internal.binary.noncompact.BinaryObjectBuilderNonCompactSimpleNameLowerCaseMappersSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesDefaultMappersSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreBinariesSimpleNameMappersSelfTest;
+import org.apache.ignite.internal.binary.streams.BinaryHeapStreamByteOrderSelfTest;
+import org.apache.ignite.internal.binary.streams.BinaryOffheapStreamByteOrderSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheBinaryStoreObjectsSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheClientNodeBinaryObjectMetadataMultinodeTest;
 import org.apache.ignite.internal.processors.cache.binary.GridCacheClientNodeBinaryObjectMetadataTest;
@@ -132,6 +134,10 @@ public class IgniteBinaryObjectsTestSuite extends TestSuite {
         suite.addTestSuite(BinaryTxCacheLocalEntriesSelfTest.class);
         suite.addTestSuite(BinaryAtomicCacheLocalEntriesSelfTest.class);
 
+        // Byte order
+        suite.addTestSuite(BinaryHeapStreamByteOrderSelfTest.class);
+        suite.addTestSuite(BinaryOffheapStreamByteOrderSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
index 1065fbd..e4a2bee 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java
@@ -19,6 +19,8 @@ package org.apache.ignite.testsuites;
 
 import java.util.Set;
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.direct.stream.v2.DirectByteBufferStreamImplV2ByteOrderSelfTest;
+import org.apache.ignite.internal.util.io.GridUnsafeDataInputOutputByteOrderSelfTest;
 import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest;
 import org.apache.ignite.marshaller.jdk.GridJdkMarshallerSelfTest;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshallerEnumSelfTest;
@@ -55,8 +57,10 @@ public class IgniteMarshallerSelfTestSuite extends TestSuite {
         GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, OptimizedObjectStreamSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridUnsafeDataOutputArraySizingSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, GridUnsafeDataInputOutputByteOrderSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerNodeFailoverTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerSerialPersistentFieldsSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, DirectByteBufferStreamImplV2ByteOrderSelfTest.class, ignoredTests);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/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
index 0efdf2b..b940c72 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
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCou
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipList;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -56,8 +57,6 @@ import org.apache.ignite.thread.IgniteThread;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.PARTITION_HASHMAP_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_REDUCER_NO_SORTING;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.UNSAFE;
 
 /**
  * Shuffle job.
@@ -299,7 +298,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
 
         /** */
         @Override public void copyTo(long ptr) {
-            UNSAFE.copyMemory(buf, BYTE_ARR_OFF + off, null, ptr, size);
+            GridUnsafe.copyMemory(buf, GridUnsafe.BYTE_ARR_OFF + off, null, ptr, size);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/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
index 87a0ee0..69dfe64 100644
--- 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
@@ -24,13 +24,11 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.UNSAFE;
-
 /**
  * Shuffle message.
  */
@@ -167,11 +165,11 @@ public class HadoopShuffleMessage implements HadoopMessage {
     private void add(byte marker, long ptr, int size) {
         buf[off++] = marker;
 
-        UNSAFE.putInt(buf, BYTE_ARR_OFF + off, size);
+        GridUnsafe.putInt(buf, GridUnsafe.BYTE_ARR_OFF + off, size);
 
         off += 4;
 
-        UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF + off, size);
+        GridUnsafe.copyMemory(null, ptr, buf, GridUnsafe.BYTE_ARR_OFF + off, size);
 
         off += size;
     }
@@ -183,7 +181,7 @@ public class HadoopShuffleMessage implements HadoopMessage {
         for (int i = 0; i < off;) {
             byte marker = buf[i++];
 
-            int size = UNSAFE.getInt(buf, BYTE_ARR_OFF + i);
+            int size = GridUnsafe.getInt(buf, GridUnsafe.BYTE_ARR_OFF + i);
 
             i += 4;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/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
index 99d8963..f7b1a73 100644
--- 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
@@ -20,11 +20,9 @@ package org.apache.ignite.internal.processors.hadoop.shuffle.streams;
 import java.io.DataOutput;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.UNSAFE;
-
 /**
  * Data output stream.
  */
@@ -69,7 +67,7 @@ public class HadoopDataOutStream extends OutputStream implements DataOutput {
 
     /** {@inheritDoc} */
     @Override public void write(byte[] b, int off, int len) {
-        UNSAFE.copyMemory(b, BYTE_ARR_OFF + off, null, move(len), len);
+        GridUnsafe.copyMemory(b, GridUnsafe.BYTE_ARR_OFF + off, null, move(len), len);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java
index d5deaac..a37d74b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java
@@ -36,14 +36,12 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
 import org.apache.ignite.internal.util.GridRandom;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.io.GridDataInput;
 import org.apache.ignite.internal.util.io.GridUnsafeDataInput;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.X;
 
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.UNSAFE;
-
 /**
  *
  */
@@ -162,7 +160,7 @@ public class HadoopConcurrentHashMultimapSelftest extends HadoopAbstractMapTest
             private void read(long ptr, int size, Writable w) {
                 assert size == 4 : size;
 
-                UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF, size);
+                GridUnsafe.copyMemory(null, ptr, buf, GridUnsafe.BYTE_ARR_OFF, size);
 
                 dataInput.bytes(buf, size);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/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
index 969eeab..f70ef2f 100644
--- 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
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
 import org.apache.ignite.internal.util.GridRandom;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.io.GridDataInput;
 import org.apache.ignite.internal.util.io.GridUnsafeDataInput;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
@@ -45,8 +46,6 @@ import org.apache.ignite.internal.util.typedef.X;
 import static java.lang.Math.abs;
 import static java.lang.Math.ceil;
 import static java.lang.Math.max;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.UNSAFE;
 
 /**
  * Skip list tests.
@@ -201,7 +200,7 @@ public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
             private void read(long ptr, int size, Writable w) {
                 assert size == 4 : size;
 
-                UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF, size);
+                GridUnsafe.copyMemory(null, ptr, buf, GridUnsafe.BYTE_ARR_OFF, size);
 
                 dataInput.bytes(buf, size);