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/10 12:56:00 UTC

[13/50] [abbrv] ignite git commit: ignite-2080 Data alignment issues with Unsafe

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
index 1a4c4bb..7958793 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java
@@ -38,41 +38,22 @@ import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemTy
 import org.apache.ignite.plugin.extensions.communication.MessageFactory;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import sun.misc.Unsafe;
 import sun.nio.ch.DirectBuffer;
 
+import static org.apache.ignite.internal.util.GridUnsafe.BIG_ENDIAN;
+import static org.apache.ignite.internal.util.GridUnsafe.BYTE_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.CHAR_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.DOUBLE_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.FLOAT_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.INT_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.LONG_ARR_OFF;
+import static org.apache.ignite.internal.util.GridUnsafe.SHORT_ARR_OFF;
+
 /**
  * Direct marshalling I/O stream (version 2).
  */
 public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** */
-    private static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
-
-    /** */
-    private static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
-
-    /** */
-    private static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
-
-    /** */
-    private static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
-
-    /** */
-    private static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
-
-    /** */
-    private static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
-
-    /** */
-    private static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
-
-    /** */
     private static final byte[] BYTE_ARR_EMPTY = new byte[0];
 
     /** */
@@ -343,7 +324,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putByte(heapArr, baseOff + pos, val);
+            GridUnsafe.putByte(heapArr, baseOff + pos, val);
 
             buf.position(pos + 1);
         }
@@ -356,7 +337,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putShort(heapArr, baseOff + pos, val);
+            long off = baseOff + pos;
+
+            if (BIG_ENDIAN)
+                GridUnsafe.putShortLE(heapArr, off, val);
+            else
+                GridUnsafe.putShort(heapArr, off, val);
 
             buf.position(pos + 2);
         }
@@ -377,12 +363,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
             while ((val & 0xFFFF_FF80) != 0) {
                 byte b = (byte)(val | 0x80);
 
-                UNSAFE.putByte(heapArr, baseOff + pos++, b);
+                GridUnsafe.putByte(heapArr, baseOff + pos++, b);
 
                 val >>>= 7;
             }
 
-            UNSAFE.putByte(heapArr, baseOff + pos++, (byte)val);
+            GridUnsafe.putByte(heapArr, baseOff + pos++, (byte)val);
 
             buf.position(pos);
         }
@@ -403,12 +389,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
             while ((val & 0xFFFF_FFFF_FFFF_FF80L) != 0) {
                 byte b = (byte)(val | 0x80);
 
-                UNSAFE.putByte(heapArr, baseOff + pos++, b);
+                GridUnsafe.putByte(heapArr, baseOff + pos++, b);
 
                 val >>>= 7;
             }
 
-            UNSAFE.putByte(heapArr, baseOff + pos++, (byte)val);
+            GridUnsafe.putByte(heapArr, baseOff + pos++, (byte)val);
 
             buf.position(pos);
         }
@@ -421,7 +407,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putFloat(heapArr, baseOff + pos, val);
+            long off = baseOff + pos;
+
+            if (BIG_ENDIAN)
+                GridUnsafe.putFloatLE(heapArr, off, val);
+            else
+                GridUnsafe.putFloat(heapArr, off, val);
 
             buf.position(pos + 4);
         }
@@ -434,7 +425,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putDouble(heapArr, baseOff + pos, val);
+            long off = baseOff + pos;
+
+            if (BIG_ENDIAN)
+                GridUnsafe.putDoubleLE(heapArr, off, val);
+            else
+                GridUnsafe.putDouble(heapArr, off, val);
 
             buf.position(pos + 8);
         }
@@ -447,7 +443,12 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putChar(heapArr, baseOff + pos, val);
+            long off = baseOff + pos;
+
+            if (BIG_ENDIAN)
+                GridUnsafe.putCharLE(heapArr, off, val);
+            else
+                GridUnsafe.putChar(heapArr, off, val);
 
             buf.position(pos + 2);
         }
@@ -460,7 +461,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         if (lastFinished) {
             int pos = buf.position();
 
-            UNSAFE.putBoolean(heapArr, baseOff + pos, val);
+            GridUnsafe.putBoolean(heapArr, baseOff + pos, val);
 
             buf.position(pos + 1);
         }
@@ -485,7 +486,10 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeShortArray(short[] val) {
         if (val != null)
-            lastFinished = writeArray(val, SHORT_ARR_OFF, val.length, val.length << 1);
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, SHORT_ARR_OFF, val.length, 2, 1);
+            else
+                lastFinished = writeArray(val, SHORT_ARR_OFF, val.length, val.length << 1);
         else
             writeInt(-1);
     }
@@ -493,7 +497,10 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeIntArray(int[] val) {
         if (val != null)
-            lastFinished = writeArray(val, INT_ARR_OFF, val.length, val.length << 2);
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, INT_ARR_OFF, val.length, 4, 2);
+            else
+                lastFinished = writeArray(val, INT_ARR_OFF, val.length, val.length << 2);
         else
             writeInt(-1);
     }
@@ -501,7 +508,10 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeLongArray(long[] val) {
         if (val != null)
-            lastFinished = writeArray(val, LONG_ARR_OFF, val.length, val.length << 3);
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, LONG_ARR_OFF, val.length, 8, 3);
+            else
+                lastFinished = writeArray(val, LONG_ARR_OFF, val.length, val.length << 3);
         else
             writeInt(-1);
     }
@@ -509,7 +519,10 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeFloatArray(float[] val) {
         if (val != null)
-            lastFinished = writeArray(val, FLOAT_ARR_OFF, val.length, val.length << 2);
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, FLOAT_ARR_OFF, val.length, 4, 2);
+            else
+                lastFinished = writeArray(val, FLOAT_ARR_OFF, val.length, val.length << 2);
         else
             writeInt(-1);
     }
@@ -517,15 +530,22 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeDoubleArray(double[] val) {
         if (val != null)
-            lastFinished = writeArray(val, DOUBLE_ARR_OFF, val.length, val.length << 3);
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, DOUBLE_ARR_OFF, val.length, 8, 3);
+            else
+                lastFinished = writeArray(val, DOUBLE_ARR_OFF, val.length, val.length << 3);
         else
             writeInt(-1);
     }
 
     /** {@inheritDoc} */
     @Override public void writeCharArray(char[] val) {
-        if (val != null)
-            lastFinished = writeArray(val, CHAR_ARR_OFF, val.length, val.length << 1);
+        if (val != null) {
+            if (BIG_ENDIAN)
+                lastFinished = writeArrayLE(val, CHAR_ARR_OFF, val.length, 2, 1);
+            else
+                lastFinished = writeArray(val, CHAR_ARR_OFF, val.length, val.length << 1);
+        }
         else
             writeInt(-1);
     }
@@ -533,7 +553,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /** {@inheritDoc} */
     @Override public void writeBooleanArray(boolean[] val) {
         if (val != null)
-            lastFinished = writeArray(val, BOOLEAN_ARR_OFF, val.length, val.length);
+            lastFinished = writeArray(val, GridUnsafe.BOOLEAN_ARR_OFF, val.length, val.length);
         else
             writeInt(-1);
     }
@@ -793,7 +813,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 1);
 
-            return UNSAFE.getByte(heapArr, baseOff + pos);
+            return GridUnsafe.getByte(heapArr, baseOff + pos);
         }
         else
             return 0;
@@ -808,7 +828,9 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 2);
 
-            return UNSAFE.getShort(heapArr, baseOff + pos);
+            long off = baseOff + pos;
+
+            return BIG_ENDIAN ? GridUnsafe.getShortLE(heapArr, off) : GridUnsafe.getShort(heapArr, off);
         }
         else
             return 0;
@@ -823,7 +845,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         while (buf.hasRemaining()) {
             int pos = buf.position();
 
-            byte b = UNSAFE.getByte(heapArr, baseOff + pos);
+            byte b = GridUnsafe.getByte(heapArr, baseOff + pos);
 
             buf.position(pos + 1);
 
@@ -860,7 +882,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         while (buf.hasRemaining()) {
             int pos = buf.position();
 
-            byte b = UNSAFE.getByte(heapArr, baseOff + pos);
+            byte b = GridUnsafe.getByte(heapArr, baseOff + pos);
 
             buf.position(pos + 1);
 
@@ -897,7 +919,9 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 4);
 
-            return UNSAFE.getFloat(heapArr, baseOff + pos);
+            long off = baseOff + pos;
+
+            return BIG_ENDIAN ? GridUnsafe.getFloatLE(heapArr, off) : GridUnsafe.getFloat(heapArr, off);
         }
         else
             return 0;
@@ -912,7 +936,9 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 8);
 
-            return UNSAFE.getDouble(heapArr, baseOff + pos);
+            long off = baseOff + pos;
+
+            return BIG_ENDIAN ? GridUnsafe.getDoubleLE(heapArr, off) : GridUnsafe.getDouble(heapArr, off);
         }
         else
             return 0;
@@ -927,7 +953,9 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 2);
 
-            return UNSAFE.getChar(heapArr, baseOff + pos);
+            long off = baseOff + pos;
+
+            return BIG_ENDIAN ? GridUnsafe.getCharLE(heapArr, off) : GridUnsafe.getChar(heapArr, off);
         }
         else
             return 0;
@@ -942,7 +970,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
             buf.position(pos + 1);
 
-            return UNSAFE.getBoolean(heapArr, baseOff + pos);
+            return GridUnsafe.getBoolean(heapArr, baseOff + pos);
         }
         else
             return false;
@@ -955,37 +983,55 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
     /** {@inheritDoc} */
     @Override public short[] readShortArray() {
-        return readArray(SHORT_ARR_CREATOR, 1, SHORT_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(SHORT_ARR_CREATOR, 2, 1, SHORT_ARR_OFF);
+        else
+            return readArray(SHORT_ARR_CREATOR, 1, SHORT_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public int[] readIntArray() {
-        return readArray(INT_ARR_CREATOR, 2, INT_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(INT_ARR_CREATOR, 4, 2, INT_ARR_OFF);
+        else
+            return readArray(INT_ARR_CREATOR, 2, INT_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public long[] readLongArray() {
-        return readArray(LONG_ARR_CREATOR, 3, LONG_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(LONG_ARR_CREATOR, 8, 3, LONG_ARR_OFF);
+        else
+            return readArray(LONG_ARR_CREATOR, 3, LONG_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public float[] readFloatArray() {
-        return readArray(FLOAT_ARR_CREATOR, 2, FLOAT_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(FLOAT_ARR_CREATOR, 4, 2, FLOAT_ARR_OFF);
+        else
+            return readArray(FLOAT_ARR_CREATOR, 2, FLOAT_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public double[] readDoubleArray() {
-        return readArray(DOUBLE_ARR_CREATOR, 3, DOUBLE_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(DOUBLE_ARR_CREATOR, 8, 3, DOUBLE_ARR_OFF);
+        else
+            return readArray(DOUBLE_ARR_CREATOR, 3, DOUBLE_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public char[] readCharArray() {
-        return readArray(CHAR_ARR_CREATOR, 1, CHAR_ARR_OFF);
+        if (BIG_ENDIAN)
+            return readArrayLE(CHAR_ARR_CREATOR, 2, 1, CHAR_ARR_OFF);
+        else
+            return readArray(CHAR_ARR_CREATOR, 1, CHAR_ARR_OFF);
     }
 
     /** {@inheritDoc} */
     @Override public boolean[] readBooleanArray() {
-        return readArray(BOOLEAN_ARR_CREATOR, 0, BOOLEAN_ARR_OFF);
+        return readArray(BOOLEAN_ARR_CREATOR, 0, GridUnsafe.BOOLEAN_ARR_OFF);
     }
 
     /** {@inheritDoc} */
@@ -1274,14 +1320,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         assert bytes >= 0;
         assert bytes >= arrOff;
 
-        if (arrOff == -1) {
-            writeInt(len);
-
-            if (!lastFinished)
-                return false;
-
-            arrOff = 0;
-        }
+        if (writeArrayLength(len))
+            return false;
 
         int toWrite = bytes - arrOff;
         int pos = buf.position();
@@ -1289,7 +1329,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
 
         if (toWrite <= remaining) {
             if (toWrite > 0) {
-                UNSAFE.copyMemory(arr, off + arrOff, heapArr, baseOff + pos, toWrite);
+                GridUnsafe.copyMemory(arr, off + arrOff, heapArr, baseOff + pos, toWrite);
 
                 buf.position(pos + toWrite);
             }
@@ -1300,7 +1340,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         }
         else {
             if (remaining > 0) {
-                UNSAFE.copyMemory(arr, off + arrOff, heapArr, baseOff + pos, remaining);
+                GridUnsafe.copyMemory(arr, off + arrOff, heapArr, baseOff + pos, remaining);
 
                 buf.position(pos + remaining);
 
@@ -1312,6 +1352,80 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     }
 
     /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param len Length.
+     * @param typeSize Primitive type size in bytes. Needs for byte reverse.
+     * @return Whether array was fully written.
+     */
+    private boolean writeArrayLE(Object arr, long off, int len, int typeSize, int shiftCnt) {
+        assert arr != null;
+        assert arr.getClass().isArray() && arr.getClass().getComponentType().isPrimitive();
+        assert off > 0;
+        assert len >= 0;
+
+        int bytes = len << shiftCnt;
+
+        assert bytes >= arrOff;
+
+        if (writeArrayLength(len))
+            return false;
+
+        int toWrite = (bytes - arrOff) >> shiftCnt;
+        int remaining = buf.remaining() >> shiftCnt;
+
+        if (toWrite <= remaining) {
+            writeArrayLE(arr, off, toWrite, typeSize);
+
+            arrOff = -1;
+
+            return true;
+        }
+        else {
+            if (remaining > 0)
+                writeArrayLE(arr, off, remaining, typeSize);
+
+            return false;
+        }
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param len Length.
+     * @param typeSize Primitive type size in bytes.
+     */
+    private void writeArrayLE(Object arr, long off, int len, int typeSize) {
+        int pos = buf.position();
+
+        for (int i = 0; i < len; i++) {
+            for (int j = 0; j < typeSize; j++) {
+                byte b = GridUnsafe.getByteField(arr, off + arrOff + (typeSize - j - 1));
+
+                GridUnsafe.putByte(heapArr, baseOff + pos++, b);
+            }
+
+            buf.position(pos);
+            arrOff += typeSize;
+        }
+    }
+
+    /**
+     * @param len Length.
+     */
+    private boolean writeArrayLength(int len) {
+        if (arrOff == -1) {
+            writeInt(len);
+
+            if (!lastFinished)
+                return true;
+
+            arrOff = 0;
+        }
+        return false;
+    }
+
+    /**
      * @param creator Array creator.
      * @param lenShift Array length shift size.
      * @param off Base offset.
@@ -1351,7 +1465,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
         lastFinished = toRead <= remaining;
 
         if (lastFinished) {
-            UNSAFE.copyMemory(heapArr, baseOff + pos, tmpArr, off + tmpArrOff, toRead);
+            GridUnsafe.copyMemory(heapArr, baseOff + pos, tmpArr, off + tmpArrOff, toRead);
 
             buf.position(pos + toRead);
 
@@ -1364,7 +1478,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
             return arr;
         }
         else {
-            UNSAFE.copyMemory(heapArr, baseOff + pos, tmpArr, off + tmpArrOff, remaining);
+            GridUnsafe.copyMemory(heapArr, baseOff + pos, tmpArr, off + tmpArrOff, remaining);
 
             buf.position(pos + remaining);
 
@@ -1375,6 +1489,95 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     }
 
     /**
+     * @param creator Array creator.
+     * @param typeSize Primitive type size in bytes.
+     * @param lenShift Array length shift size.
+     * @param off Base offset.
+     * @return Array or special value if it was not fully read.
+     */
+    @SuppressWarnings("unchecked")
+    private <T> T readArrayLE(ArrayCreator<T> creator, int typeSize, int lenShift, long off) {
+        assert creator != null;
+
+        if (tmpArr == null) {
+            int len = readInt();
+
+            if (!lastFinished)
+                return null;
+
+            switch (len) {
+                case -1:
+                    lastFinished = true;
+
+                    return null;
+
+                case 0:
+                    lastFinished = true;
+
+                    return creator.create(0);
+
+                default:
+                    tmpArr = creator.create(len);
+                    tmpArrBytes = len << lenShift;
+            }
+        }
+
+        int toRead = (tmpArrBytes - tmpArrOff) >> lenShift;
+        int remaining = buf.remaining() >> lenShift;
+
+        lastFinished = toRead <= buf.remaining();
+
+        if (lastFinished) {
+            readArrayLE(typeSize, off, toRead);
+
+            T arr = (T)tmpArr;
+
+            tmpArr = null;
+            tmpArrBytes = 0;
+            tmpArrOff = 0;
+
+            return arr;
+        }
+        else {
+            for (int i = 0; i < remaining; i++) {
+                int pos = buf.position();
+
+                for (int j = 0; j < typeSize; j++) {
+                    byte b = GridUnsafe.getByte(heapArr, baseOff + pos + (typeSize - j - 1));
+
+                    GridUnsafe.putByteField(tmpArr, off + tmpArrOff + j, b);
+                }
+
+                buf.position(pos + typeSize);
+                tmpArrOff += typeSize;
+            }
+
+            tmpArrOff += buf.remaining();
+
+            return null;
+        }
+    }
+
+    /**
+     * @param typeSize Primitive type size in bytes.
+     * @param off Offset.
+     * @param toRead To read.
+     */
+    private void readArrayLE(int typeSize, long off, int toRead) {
+        for (int i = 0; i < toRead; i++) {
+            int pos = buf.position();
+
+            for (int j = 0; j < typeSize; j++) {
+                byte b = GridUnsafe.getByte(heapArr, baseOff + pos + (typeSize - j - 1));
+
+                GridUnsafe.putByteField(tmpArr, off + tmpArrOff++, b);
+            }
+
+            buf.position(pos + typeSize);
+        }
+    }
+
+    /**
      * @param type Type.
      * @param val Value.
      * @param writer Writer.
@@ -1583,7 +1786,7 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream {
     /**
      * Array creator.
      */
-    private static interface ArrayCreator<T> {
+    private interface ArrayCreator<T> {
         /**
          * @param len Array length or {@code -1} if array was not fully read.
          * @return New array.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 845e204..3a7bc8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -103,9 +103,6 @@ import static org.jsr166.ConcurrentLinkedDeque8.Node;
  * Cache eviction manager.
  */
 public class GridCacheEvictionManager extends GridCacheManagerAdapter {
-    /** Unsafe instance. */
-    private static final sun.misc.Unsafe unsafe = GridUnsafe.unsafe();
-
     /** Attribute name used to queue node in entry metadata. */
     private static final int META_KEY = GridMetadataAwareAdapter.EntryKey.CACHE_EVICTION_MANAGER_KEY.key();
 
@@ -985,7 +982,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                     continue;
 
                 // Lock entry.
-                unsafe.monitorEnter(entry);
+                GridUnsafe.monitorEnter(entry);
 
                 locked.add(entry);
 
@@ -1028,7 +1025,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
             for (ListIterator<GridCacheEntryEx> it = locked.listIterator(locked.size()); it.hasPrevious();) {
                 GridCacheEntryEx e = it.previous();
 
-                unsafe.monitorExit(e);
+                GridUnsafe.monitorExit(e);
             }
 
             // Remove entries and fire events outside the locks.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapSwapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapSwapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapSwapEntry.java
index ea036af..82e115c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapSwapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapSwapEntry.java
@@ -23,7 +23,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
-import sun.misc.Unsafe;
 
 /**
  * GridCacheSwapEntry over offheap pointer.
@@ -41,9 +40,6 @@ import sun.misc.Unsafe;
  */
 public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
     /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
     private final long ptr;
 
     /** */
@@ -70,17 +66,17 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
 
         long readPtr = ptr + GridCacheSwapEntryImpl.VERSION_OFFSET;
 
-        boolean verEx = UNSAFE.getByte(readPtr++) != 0;
+        boolean verEx = GridUnsafe.getByte(readPtr++) != 0;
 
         ver = U.readVersion(readPtr, verEx);
 
         readPtr += verEx ? GridCacheSwapEntryImpl.VERSION_EX_SIZE : GridCacheSwapEntryImpl.VERSION_SIZE;
 
-        type = UNSAFE.getByte(readPtr + 4);
+        type = GridUnsafe.getByte(readPtr + 4);
 
         valPtr = readPtr;
 
-        assert (ptr + size) > (UNSAFE.getInt(valPtr) + valPtr + 5);
+        assert (ptr + size) > (GridUnsafe.getInt(valPtr) + valPtr + 5);
     }
 
     /**
@@ -94,11 +90,11 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
 
         ptr += GridCacheSwapEntryImpl.VERSION_OFFSET; // Skip ttl, expire time.
 
-        boolean verEx = UNSAFE.getByte(ptr++) != 0;
+        boolean verEx = GridUnsafe.getByte(ptr++) != 0;
 
         ptr += verEx ? GridCacheSwapEntryImpl.VERSION_EX_SIZE : GridCacheSwapEntryImpl.VERSION_SIZE;
 
-        assert (ptr + size) > (UNSAFE.getInt(ptr) + ptr + 5);
+        assert (ptr + size) > (GridUnsafe.getInt(ptr) + ptr + 5);
 
         return ptr;
     }
@@ -108,7 +104,7 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
      * @return TTL.
      */
     public static long timeToLive(long ptr) {
-        return UNSAFE.getLong(ptr);
+        return GridUnsafe.getLong(ptr);
     }
 
     /**
@@ -116,7 +112,7 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
      * @return Expire time.
      */
     public static long expireTime(long ptr) {
-        return UNSAFE.getLong(ptr + GridCacheSwapEntryImpl.EXPIRE_TIME_OFFSET);
+        return GridUnsafe.getLong(ptr + GridCacheSwapEntryImpl.EXPIRE_TIME_OFFSET);
     }
 
     /**
@@ -126,7 +122,7 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
     public static GridCacheVersion version(long ptr) {
         long addr = ptr + GridCacheSwapEntryImpl.VERSION_OFFSET;
 
-        boolean verEx = UNSAFE.getByte(addr) != 0;
+        boolean verEx = GridUnsafe.getByte(addr) != 0;
 
         addr++;
 
@@ -165,12 +161,12 @@ public class GridCacheOffheapSwapEntry implements GridCacheSwapEntry {
 
     /** {@inheritDoc} */
     @Override public long ttl() {
-        return UNSAFE.getLong(ptr);
+        return GridUnsafe.getLong(ptr);
     }
 
     /** {@inheritDoc} */
     @Override public long expireTime() {
-        return UNSAFE.getLong(ptr + GridCacheSwapEntryImpl.EXPIRE_TIME_OFFSET);
+        return GridUnsafe.getLong(ptr + GridCacheSwapEntryImpl.EXPIRE_TIME_OFFSET);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
index 6b1266f..02f74e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapEntryImpl.java
@@ -26,19 +26,12 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
-import sun.misc.Unsafe;
 
 /**
  * Swap entry.
  */
 public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
     /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** */
     static final int EXPIRE_TIME_OFFSET = 8;
 
     /** */
@@ -108,7 +101,7 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return TTL.
      */
     public static long timeToLive(byte[] bytes) {
-        return UNSAFE.getLong(bytes, BYTE_ARR_OFF);
+        return GridUnsafe.getLong(bytes, GridUnsafe.BYTE_ARR_OFF);
     }
 
     /**
@@ -116,7 +109,7 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return Expire time.
      */
     public static long expireTime(byte[] bytes) {
-        return UNSAFE.getLong(bytes, BYTE_ARR_OFF + EXPIRE_TIME_OFFSET);
+        return GridUnsafe.getLong(bytes, GridUnsafe.BYTE_ARR_OFF + EXPIRE_TIME_OFFSET);
     }
 
     /**
@@ -124,9 +117,9 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return Version.
      */
     public static GridCacheVersion version(byte[] bytes) {
-        long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
+        long off = GridUnsafe.BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
 
-        boolean verEx = UNSAFE.getByte(bytes, off++) != 0;
+        boolean verEx = GridUnsafe.getByte(bytes, off++) != 0;
 
         return U.readVersion(bytes, off, verEx);
     }
@@ -136,21 +129,21 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      * @return Value if value is byte array, otherwise {@code null}.
      */
     @Nullable public static IgniteBiTuple<byte[], Byte> getValue(byte[] bytes) {
-        long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
+        long off = GridUnsafe.BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
 
-        boolean verEx = UNSAFE.getByte(bytes, off++) != 0;
+        boolean verEx = GridUnsafe.getByte(bytes, off++) != 0;
 
         off += verEx ? VERSION_EX_SIZE : VERSION_SIZE;
 
-        int arrLen = UNSAFE.getInt(bytes, off);
+        int arrLen = GridUnsafe.getInt(bytes, off);
 
         off += 4;
 
-        byte type = UNSAFE.getByte(bytes, off++);
+        byte type = GridUnsafe.getByte(bytes, off++);
 
         byte[] valBytes = new byte[arrLen];
 
-        UNSAFE.copyMemory(bytes, off, valBytes, BYTE_ARR_OFF, arrLen);
+        GridUnsafe.copyMemory(bytes, off, valBytes, GridUnsafe.BYTE_ARR_OFF, arrLen);
 
         return new IgniteBiTuple<>(valBytes, type);
     }
@@ -235,25 +228,25 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
 
         byte[] arr = new byte[size];
 
-        long off = BYTE_ARR_OFF;
+        long off = GridUnsafe.BYTE_ARR_OFF;
 
-        UNSAFE.putLong(arr, off, ttl);
+        GridUnsafe.putLong(arr, off, ttl);
 
         off += 8;
 
-        UNSAFE.putLong(arr, off, expireTime);
+        GridUnsafe.putLong(arr, off, expireTime);
 
         off += 8;
 
         off = U.writeVersion(arr, off, ver);
 
-        UNSAFE.putInt(arr, off, len);
+        GridUnsafe.putInt(arr, off, len);
 
         off += 4;
 
-        UNSAFE.putByte(arr, off++, type);
+        GridUnsafe.putByte(arr, off++, type);
 
-        UNSAFE.copyMemory(valBytes.array(), BYTE_ARR_OFF, arr, off, len);
+        GridUnsafe.copyMemory(valBytes.array(), GridUnsafe.BYTE_ARR_OFF, arr, off, len);
 
         off += len;
 
@@ -271,21 +264,21 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
      */
     public static GridCacheSwapEntryImpl unmarshal(byte[] arr, boolean valOnly) {
         if (valOnly) {
-            long off = BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
+            long off = GridUnsafe.BYTE_ARR_OFF + VERSION_OFFSET; // Skip ttl, expire time.
 
-            boolean verEx = UNSAFE.getByte(arr, off++) != 0;
+            boolean verEx = GridUnsafe.getByte(arr, off++) != 0;
 
             off += verEx ? VERSION_EX_SIZE : VERSION_SIZE;
 
-            int arrLen = UNSAFE.getInt(arr, off);
+            int arrLen = GridUnsafe.getInt(arr, off);
 
             off += 4;
 
-            byte type = UNSAFE.getByte(arr, off++);
+            byte type = GridUnsafe.getByte(arr, off++);
 
             byte[] valBytes = new byte[arrLen];
 
-            UNSAFE.copyMemory(arr, off, valBytes, BYTE_ARR_OFF, arrLen);
+            GridUnsafe.copyMemory(arr, off, valBytes, GridUnsafe.BYTE_ARR_OFF, arrLen);
 
             return new GridCacheSwapEntryImpl(ByteBuffer.wrap(valBytes),
                 type,
@@ -296,31 +289,31 @@ public class GridCacheSwapEntryImpl implements GridCacheSwapEntry {
                 null);
         }
 
-        long off = BYTE_ARR_OFF;
+        long off = GridUnsafe.BYTE_ARR_OFF;
 
-        long ttl = UNSAFE.getLong(arr, off);
+        long ttl = GridUnsafe.getLong(arr, off);
 
         off += 8;
 
-        long expireTime = UNSAFE.getLong(arr, off);
+        long expireTime = GridUnsafe.getLong(arr, off);
 
         off += 8;
 
-        boolean verEx = UNSAFE.getBoolean(arr, off++);
+        boolean verEx = GridUnsafe.getBoolean(arr, off++);
 
         GridCacheVersion ver = U.readVersion(arr, off, verEx);
 
         off += verEx ? VERSION_EX_SIZE : VERSION_SIZE;
 
-        int arrLen = UNSAFE.getInt(arr, off);
+        int arrLen = GridUnsafe.getInt(arr, off);
 
         off += 4;
 
-        byte type = UNSAFE.getByte(arr, off++);
+        byte type = GridUnsafe.getByte(arr, off++);
 
         byte[] valBytes = new byte[arrLen];
 
-        UNSAFE.copyMemory(arr, off, valBytes, BYTE_ARR_OFF, arrLen);
+        GridUnsafe.copyMemory(arr, off, valBytes, GridUnsafe.BYTE_ARR_OFF, arrLen);
 
         off += arrLen;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index a21331e..0fef6f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -102,7 +102,6 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
-import sun.misc.Unsafe;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
@@ -113,9 +112,6 @@ import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorImpl implements
     CacheObjectBinaryProcessor {
     /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
     public static final IgniteProductVersion BINARY_CFG_CHECK_SINCE = IgniteProductVersion.fromString("1.5.6");
 
     /** */
@@ -430,11 +426,11 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectException {
         assert ptr > 0 : ptr;
 
-        int size = UNSAFE.getInt(ptr);
+        int size = GridUnsafe.getInt(ptr);
 
         ptr += 4;
 
-        byte type = UNSAFE.getByte(ptr++);
+        byte type = GridUnsafe.getByte(ptr++);
 
         if (type != CacheObject.TYPE_BYTE_ARR) {
             assert size > 0 : size;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index b806906..f6f57ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -138,9 +138,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     private static final int DEFERRED_UPDATE_RESPONSE_TIMEOUT =
         Integer.getInteger(IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT, 500);
 
-    /** Unsafe instance. */
-    private static final sun.misc.Unsafe UNSAFE = GridUnsafe.unsafe();
-
     /** Update reply closure. */
     private CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse> updateReplyClos;
 
@@ -2463,10 +2460,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 try {
                     GridDhtCacheEntry entry = entryExx(key, topVer);
 
-                    UNSAFE.monitorEnter(entry);
+                    GridUnsafe.monitorEnter(entry);
 
                     if (entry.obsolete())
-                        UNSAFE.monitorExit(entry);
+                        GridUnsafe.monitorExit(entry);
                     else
                         return Collections.singletonList(entry);
                 }
@@ -2506,13 +2503,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     if (entry == null)
                         continue;
 
-                    UNSAFE.monitorEnter(entry);
+                    GridUnsafe.monitorEnter(entry);
 
                     if (entry.obsolete()) {
                         // Unlock all locked.
                         for (int j = 0; j <= i; j++) {
                             if (locked.get(j) != null)
-                                UNSAFE.monitorExit(locked.get(j));
+                                GridUnsafe.monitorExit(locked.get(j));
                         }
 
                         // Clear entries.
@@ -2561,7 +2558,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             // That's why releasing locks in the finally block..
             for (GridCacheMapEntry entry : locked) {
                 if (entry != null)
-                    UNSAFE.monitorExit(entry);
+                    GridUnsafe.monitorExit(entry);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index fed3e33..8e5fe9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -89,9 +89,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Unsafe instance. */
-    private static final sun.misc.Unsafe UNSAFE = GridUnsafe.unsafe();
-
     /** */
     private GridCachePreloader preldr;
 
@@ -1553,12 +1550,12 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             for (int i = 0; i < locked.size(); i++) {
                 GridCacheEntryEx entry = locked.get(i);
 
-                UNSAFE.monitorEnter(entry);
+                GridUnsafe.monitorEnter(entry);
 
                 if (entry.obsolete()) {
                     // Unlock all locked.
                     for (int j = 0; j <= i; j++)
-                        UNSAFE.monitorExit(locked.get(j));
+                        GridUnsafe.monitorExit(locked.get(j));
 
                     // Clear entries.
                     locked.clear();
@@ -1589,7 +1586,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      */
     private void unlockEntries(Iterable<GridCacheEntryEx> locked) {
         for (GridCacheEntryEx entry : locked)
-            UNSAFE.monitorExit(entry);
+            GridUnsafe.monitorExit(entry);
 
         AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 5b764b6..54dd69e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -51,9 +51,6 @@ import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
  *
  */
 public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter implements IgniteCacheObjectProcessor {
-    /** */
-    private static final sun.misc.Unsafe UNSAFE = GridUnsafe.unsafe();
-
     /** Immutable classes. */
     private static final Collection<Class<?>> IMMUTABLE_CLS = new HashSet<>();
 
@@ -138,9 +135,9 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     {
         assert valPtr != 0;
 
-        int size = UNSAFE.getInt(valPtr);
+        int size = GridUnsafe.getInt(valPtr);
 
-        byte type = UNSAFE.getByte(valPtr + 4);
+        byte type = GridUnsafe.getByte(valPtr + 4);
 
         byte[] bytes = U.copyMemory(valPtr + 5, size);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformAbstractMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformAbstractMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformAbstractMemory.java
index e305c71..606a23c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformAbstractMemory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformAbstractMemory.java
@@ -17,13 +17,15 @@
 
 package org.apache.ignite.internal.processors.platform.memory;
 
+import org.apache.ignite.internal.util.GridUnsafe;
+
 /**
  * Interop memory chunk abstraction.
  */
 public abstract class PlatformAbstractMemory implements PlatformMemory {
     /** Stream factory. */
-    private static final StreamFactory STREAM_FACTORY = PlatformMemoryUtils.LITTLE_ENDIAN ?
-        new LittleEndianStreamFactory() : new BigEndianStreamFactory();
+    private static final StreamFactory STREAM_FACTORY = GridUnsafe.BIG_ENDIAN ?
+        new BigEndianStreamFactory() : new LittleEndianStreamFactory();
 
     /** Cross-platform memory pointer. */
     protected long memPtr;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianOutputStreamImpl.java
index 2f6ad5c..b16f42f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformBigEndianOutputStreamImpl.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.memory;
 
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.UNSAFE;
+import org.apache.ignite.internal.util.GridUnsafe;
 
 /**
  * Interop output stream implementation working with BIG ENDIAN architecture.
@@ -46,7 +46,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (short item : val) {
-            UNSAFE.putShort(startPos, Short.reverseBytes(item));
+            GridUnsafe.putShort(startPos, Short.reverseBytes(item));
 
             startPos += 2;
         }
@@ -68,7 +68,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (char item : val) {
-            UNSAFE.putChar(startPos, Character.reverseBytes(item));
+            GridUnsafe.putChar(startPos, Character.reverseBytes(item));
 
             startPos += 2;
         }
@@ -90,7 +90,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (int item : val) {
-            UNSAFE.putInt(startPos, Integer.reverseBytes(item));
+            GridUnsafe.putInt(startPos, Integer.reverseBytes(item));
 
             startPos += 4;
         }
@@ -117,7 +117,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (float item : val) {
-            UNSAFE.putInt(startPos, Integer.reverseBytes(Float.floatToIntBits(item)));
+            GridUnsafe.putInt(startPos, Integer.reverseBytes(Float.floatToIntBits(item)));
 
             startPos += 4;
         }
@@ -139,7 +139,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (long item : val) {
-            UNSAFE.putLong(startPos, Long.reverseBytes(item));
+            GridUnsafe.putLong(startPos, Long.reverseBytes(item));
 
             startPos += 8;
         }
@@ -156,7 +156,7 @@ public class PlatformBigEndianOutputStreamImpl extends PlatformOutputStreamImpl
         long startPos = data + pos;
 
         for (double item : val) {
-            UNSAFE.putLong(startPos, Long.reverseBytes(Double.doubleToLongBits(item)));
+            GridUnsafe.putLong(startPos, Long.reverseBytes(Double.doubleToLongBits(item)));
 
             startPos += 8;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
index 13da8c5..5e26905 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
@@ -18,16 +18,7 @@
 package org.apache.ignite.internal.processors.platform.memory;
 
 import org.apache.ignite.IgniteException;
-
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.BOOLEAN_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.CHAR_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.DOUBLE_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.FLOAT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.INT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.LONG_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.SHORT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.UNSAFE;
+import org.apache.ignite.internal.util.GridUnsafe;
 
 /**
  * Interop input stream implementation.
@@ -64,14 +55,14 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public byte readByte() {
         ensureEnoughData(1);
 
-        return UNSAFE.getByte(data + pos++);
+        return GridUnsafe.getByte(data + pos++);
     }
 
     /** {@inheritDoc} */
     @Override public byte[] readByteArray(int cnt) {
         byte[] res = new byte[cnt];
 
-        copyAndShift(res, BYTE_ARR_OFF, cnt);
+        copyAndShift(res, GridUnsafe.BYTE_ARR_OFF, cnt);
 
         return res;
     }
@@ -85,7 +76,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public boolean[] readBooleanArray(int cnt) {
         boolean[] res = new boolean[cnt];
 
-        copyAndShift(res, BOOLEAN_ARR_OFF, cnt);
+        copyAndShift(res, GridUnsafe.BOOLEAN_ARR_OFF, cnt);
 
         return res;
     }
@@ -94,7 +85,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public short readShort() {
         ensureEnoughData(2);
 
-        short res = UNSAFE.getShort(data + pos);
+        short res = GridUnsafe.getShort(data + pos);
 
         shift(2);
 
@@ -107,7 +98,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         short[] res = new short[cnt];
 
-        copyAndShift(res, SHORT_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.SHORT_ARR_OFF, len);
 
         return res;
     }
@@ -116,7 +107,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public char readChar() {
         ensureEnoughData(2);
 
-        char res = UNSAFE.getChar(data + pos);
+        char res = GridUnsafe.getChar(data + pos);
 
         shift(2);
 
@@ -129,7 +120,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         char[] res = new char[cnt];
 
-        copyAndShift(res, CHAR_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.CHAR_ARR_OFF, len);
 
         return res;
     }
@@ -138,7 +129,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public int readInt() {
         ensureEnoughData(4);
 
-        int res = UNSAFE.getInt(data + pos);
+        int res = GridUnsafe.getInt(data + pos);
 
         shift(4);
 
@@ -152,7 +143,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
         if (delta > 0)
             ensureEnoughData(delta);
 
-        return UNSAFE.getByte(data + pos);
+        return GridUnsafe.getByte(data + pos);
     }
 
     /** {@inheritDoc} */
@@ -162,7 +153,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
         if (delta > 0)
             ensureEnoughData(delta);
 
-        return UNSAFE.getShort(data + pos);
+        return GridUnsafe.getShort(data + pos);
     }
 
     /** {@inheritDoc} */
@@ -172,7 +163,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
         if (delta > 0)
             ensureEnoughData(delta);
 
-        return UNSAFE.getInt(data + pos);
+        return GridUnsafe.getInt(data + pos);
     }
 
     /** {@inheritDoc} */
@@ -181,7 +172,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         int[] res = new int[cnt];
 
-        copyAndShift(res, INT_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.INT_ARR_OFF, len);
 
         return res;
     }
@@ -190,7 +181,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public float readFloat() {
         ensureEnoughData(4);
 
-        float res = UNSAFE.getFloat(data + pos);
+        float res = GridUnsafe.getFloat(data + pos);
 
         shift(4);
 
@@ -203,7 +194,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         float[] res = new float[cnt];
 
-        copyAndShift(res, FLOAT_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.FLOAT_ARR_OFF, len);
 
         return res;
     }
@@ -212,7 +203,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public long readLong() {
         ensureEnoughData(8);
 
-        long res = UNSAFE.getLong(data + pos);
+        long res = GridUnsafe.getLong(data + pos);
 
         shift(8);
 
@@ -225,7 +216,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         long[] res = new long[cnt];
 
-        copyAndShift(res, LONG_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.LONG_ARR_OFF, len);
 
         return res;
     }
@@ -234,7 +225,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     @Override public double readDouble() {
         ensureEnoughData(8);
 
-        double res = UNSAFE.getDouble(data + pos);
+        double res = GridUnsafe.getDouble(data + pos);
 
         shift(8);
 
@@ -247,7 +238,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
 
         double[] res = new double[cnt];
 
-        copyAndShift(res, DOUBLE_ARR_OFF, len);
+        copyAndShift(res, GridUnsafe.DOUBLE_ARR_OFF, len);
 
         return res;
     }
@@ -257,7 +248,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
         if (len > remaining())
             len = remaining();
 
-        copyAndShift(arr, BYTE_ARR_OFF + off, len);
+        copyAndShift(arr, GridUnsafe.BYTE_ARR_OFF + off, len);
 
         return len;
     }
@@ -290,7 +281,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
         if (dataCopy == null) {
             dataCopy = new byte[len];
 
-            UNSAFE.copyMemory(null, data, dataCopy, BYTE_ARR_OFF, dataCopy.length);
+            GridUnsafe.copyMemory(null, data, dataCopy, GridUnsafe.BYTE_ARR_OFF, dataCopy.length);
         }
 
         return dataCopy;
@@ -333,7 +324,7 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     private void copyAndShift(Object target, long off, int cnt) {
         ensureEnoughData(cnt);
 
-        UNSAFE.copyMemory(null, data + pos, target, off, cnt);
+        GridUnsafe.copyMemory(null, data + pos, target, off, cnt);
 
         shift(cnt);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryUtils.java
index 2520a47..894658c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryUtils.java
@@ -17,44 +17,12 @@
 
 package org.apache.ignite.internal.processors.platform.memory;
 
-import java.nio.ByteOrder;
 import org.apache.ignite.internal.util.GridUnsafe;
-import sun.misc.Unsafe;
 
 /**
  * Utility classes for memory management.
  */
 public class PlatformMemoryUtils {
-    /** Unsafe instance. */
-    public static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** Array offset: boolean. */
-    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
-
-    /** Array offset: byte. */
-    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** Array offset: short. */
-    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
-
-    /** Array offset: char. */
-    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
-
-    /** Array offset: int. */
-    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
-
-    /** Array offset: float. */
-    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
-
-    /** Array offset: long. */
-    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
-
-    /** Array offset: double. */
-    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
-
-    /** Whether little endian is used on the platform. */
-    public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
-
     /** Header length. */
     public static final int POOL_HDR_LEN = 64;
 
@@ -97,7 +65,7 @@ public class PlatformMemoryUtils {
      * @return Data pointer.
      */
     public static long data(long memPtr) {
-        return UNSAFE.getLong(memPtr);
+        return GridUnsafe.getLong(memPtr);
     }
 
     /**
@@ -107,7 +75,7 @@ public class PlatformMemoryUtils {
      * @return Capacity.
      */
     public static int capacity(long memPtr) {
-        return UNSAFE.getInt(memPtr + MEM_HDR_OFF_CAP);
+        return GridUnsafe.getInt(memPtr + MEM_HDR_OFF_CAP);
     }
 
     /**
@@ -119,7 +87,7 @@ public class PlatformMemoryUtils {
     public static void capacity(long memPtr, int cap) {
         assert !isExternal(memPtr) : "Attempt to update external memory chunk capacity: " + memPtr;
 
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
     }
 
     /**
@@ -129,7 +97,7 @@ public class PlatformMemoryUtils {
      * @return Length.
      */
     public static int length(long memPtr) {
-        return UNSAFE.getInt(memPtr + MEM_HDR_OFF_LEN);
+        return GridUnsafe.getInt(memPtr + MEM_HDR_OFF_LEN);
     }
 
     /**
@@ -139,7 +107,7 @@ public class PlatformMemoryUtils {
      * @param len Length.
      */
     public static void length(long memPtr, int len) {
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_LEN, len);
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_LEN, len);
     }
 
     /**
@@ -149,7 +117,7 @@ public class PlatformMemoryUtils {
      * @return Flags.
      */
     public static int flags(long memPtr) {
-        return UNSAFE.getInt(memPtr + MEM_HDR_OFF_FLAGS);
+        return GridUnsafe.getInt(memPtr + MEM_HDR_OFF_FLAGS);
     }
 
     /**
@@ -161,7 +129,7 @@ public class PlatformMemoryUtils {
     public static void flags(long memPtr, int flags) {
         assert !isExternal(memPtr) : "Attempt to update external memory chunk flags: " + memPtr;
 
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_FLAGS, flags);
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_FLAGS, flags);
     }
 
     /**
@@ -237,13 +205,13 @@ public class PlatformMemoryUtils {
     public static long allocateUnpooled(int cap) {
         assert cap > 0;
 
-        long memPtr = UNSAFE.allocateMemory(MEM_HDR_LEN);
-        long dataPtr = UNSAFE.allocateMemory(cap);
+        long memPtr = GridUnsafe.allocateMemory(MEM_HDR_LEN);
+        long dataPtr = GridUnsafe.allocateMemory(cap);
 
-        UNSAFE.putLong(memPtr, dataPtr);              // Write address.
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap); // Write capacity.
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_LEN, 0);   // Write length.
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_FLAGS, 0); // Write flags.
+        GridUnsafe.putLong(memPtr, dataPtr);              // Write address.
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap); // Write capacity.
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_LEN, 0);   // Write length.
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_FLAGS, 0); // Write flags.
 
         return memPtr;
     }
@@ -262,12 +230,12 @@ public class PlatformMemoryUtils {
 
         long dataPtr = data(memPtr);
 
-        long newDataPtr = UNSAFE.reallocateMemory(dataPtr, cap);
+        long newDataPtr = GridUnsafe.reallocateMemory(dataPtr, cap);
 
         if (dataPtr != newDataPtr)
-            UNSAFE.putLong(memPtr, newDataPtr); // Write new data address if needed.
+            GridUnsafe.putLong(memPtr, newDataPtr); // Write new data address if needed.
 
-        UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap); // Write new capacity.
+        GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap); // Write new capacity.
     }
 
     /**
@@ -279,8 +247,8 @@ public class PlatformMemoryUtils {
         assert !isExternal(memPtr) : "Attempt to release external memory chunk directly: " + memPtr;
         assert !isPooled(memPtr) : "Attempt to release pooled memory chunk directly: " + memPtr;
 
-        UNSAFE.freeMemory(data(memPtr));
-        UNSAFE.freeMemory(memPtr);
+        GridUnsafe.freeMemory(data(memPtr));
+        GridUnsafe.freeMemory(memPtr);
     }
 
     /** --- POOLED MEMORY MANAGEMENT. --- */
@@ -291,9 +259,9 @@ public class PlatformMemoryUtils {
      * @return Pool pointer.
      */
     public static long allocatePool() {
-        long poolPtr = UNSAFE.allocateMemory(POOL_HDR_LEN);
+        long poolPtr = GridUnsafe.allocateMemory(POOL_HDR_LEN);
 
-        UNSAFE.setMemory(poolPtr, POOL_HDR_LEN, (byte)0);
+        GridUnsafe.setMemory(poolPtr, POOL_HDR_LEN, (byte)0);
 
         flags(poolPtr + POOL_HDR_OFF_MEM_1, FLAG_POOLED);
         flags(poolPtr + POOL_HDR_OFF_MEM_2, FLAG_POOLED);
@@ -309,23 +277,23 @@ public class PlatformMemoryUtils {
      */
     public static void releasePool(long poolPtr) {
         // Clean predefined memory chunks.
-        long mem = UNSAFE.getLong(poolPtr + POOL_HDR_OFF_MEM_1);
+        long mem = GridUnsafe.getLong(poolPtr + POOL_HDR_OFF_MEM_1);
 
         if (mem != 0)
-            UNSAFE.freeMemory(mem);
+            GridUnsafe.freeMemory(mem);
 
-        mem = UNSAFE.getLong(poolPtr + POOL_HDR_OFF_MEM_2);
+        mem = GridUnsafe.getLong(poolPtr + POOL_HDR_OFF_MEM_2);
 
         if (mem != 0)
-            UNSAFE.freeMemory(mem);
+            GridUnsafe.freeMemory(mem);
 
-        mem = UNSAFE.getLong(poolPtr + POOL_HDR_OFF_MEM_3);
+        mem = GridUnsafe.getLong(poolPtr + POOL_HDR_OFF_MEM_3);
 
         if (mem != 0)
-            UNSAFE.freeMemory(mem);
+            GridUnsafe.freeMemory(mem);
 
         // Clean pool chunk.
-        UNSAFE.freeMemory(poolPtr);
+        GridUnsafe.freeMemory(poolPtr);
     }
 
     /**
@@ -376,24 +344,24 @@ public class PlatformMemoryUtils {
         assert isPooled(memPtr);
         assert !isAcquired(memPtr);
 
-        long data = UNSAFE.getLong(memPtr);
+        long data = GridUnsafe.getLong(memPtr);
 
         if (data == 0) {
             // First allocation of the chunk.
-            data = UNSAFE.allocateMemory(cap);
+            data = GridUnsafe.allocateMemory(cap);
 
-            UNSAFE.putLong(memPtr, data);
-            UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
+            GridUnsafe.putLong(memPtr, data);
+            GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
         }
         else {
             // Ensure that we have enough capacity.
             int curCap = capacity(memPtr);
 
             if (cap > curCap) {
-                data = UNSAFE.reallocateMemory(data, cap);
+                data = GridUnsafe.reallocateMemory(data, cap);
 
-                UNSAFE.putLong(memPtr, data);
-                UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
+                GridUnsafe.putLong(memPtr, data);
+                GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
             }
         }
 
@@ -411,17 +379,17 @@ public class PlatformMemoryUtils {
         assert isPooled(memPtr);
         assert isAcquired(memPtr);
 
-        long data = UNSAFE.getLong(memPtr);
+        long data = GridUnsafe.getLong(memPtr);
 
         assert data != 0;
 
         int curCap = capacity(memPtr);
 
         if (cap > curCap) {
-            data = UNSAFE.reallocateMemory(data, cap);
+            data = GridUnsafe.reallocateMemory(data, cap);
 
-            UNSAFE.putLong(memPtr, data);
-            UNSAFE.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
+            GridUnsafe.putLong(memPtr, data);
+            GridUnsafe.putInt(memPtr + MEM_HDR_OFF_CAP, cap);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
index 59d8981..cb30336 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
@@ -17,15 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.memory;
 
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.BOOLEAN_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.BYTE_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.CHAR_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.DOUBLE_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.FLOAT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.INT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.LONG_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.SHORT_ARR_OFF;
-import static org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils.UNSAFE;
+import org.apache.ignite.internal.util.GridUnsafe;
 
 /**
  * Interop output stream implementation.
@@ -59,12 +51,12 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
     @Override public void writeByte(byte val) {
         ensureCapacity(pos + 1);
 
-        UNSAFE.putByte(data + pos++, val);
+        GridUnsafe.putByte(data + pos++, val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeByteArray(byte[] val) {
-        copyAndShift(val, BYTE_ARR_OFF, val.length);
+        copyAndShift(val, GridUnsafe.BYTE_ARR_OFF, val.length);
     }
 
     /** {@inheritDoc} */
@@ -74,63 +66,63 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
     /** {@inheritDoc} */
     @Override public void writeBooleanArray(boolean[] val) {
-        copyAndShift(val, BOOLEAN_ARR_OFF, val.length);
+        copyAndShift(val, GridUnsafe.BOOLEAN_ARR_OFF, val.length);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(short val) {
         ensureCapacity(pos + 2);
 
-        UNSAFE.putShort(data + pos, val);
+        GridUnsafe.putShort(data + pos, val);
 
         shift(2);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShortArray(short[] val) {
-        copyAndShift(val, SHORT_ARR_OFF, val.length << 1);
+        copyAndShift(val, GridUnsafe.SHORT_ARR_OFF, val.length << 1);
     }
 
     /** {@inheritDoc} */
     @Override public void writeChar(char val) {
         ensureCapacity(pos + 2);
 
-        UNSAFE.putChar(data + pos, val);
+        GridUnsafe.putChar(data + pos, val);
 
         shift(2);
     }
 
     /** {@inheritDoc} */
     @Override public void writeCharArray(char[] val) {
-        copyAndShift(val, CHAR_ARR_OFF, val.length << 1);
+        copyAndShift(val, GridUnsafe.CHAR_ARR_OFF, val.length << 1);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(int val) {
         ensureCapacity(pos + 4);
 
-        UNSAFE.putInt(data + pos, val);
+        GridUnsafe.putInt(data + pos, val);
 
         shift(4);
     }
 
     /** {@inheritDoc} */
     @Override public void writeIntArray(int[] val) {
-        copyAndShift(val, INT_ARR_OFF, val.length << 2);
+        copyAndShift(val, GridUnsafe.INT_ARR_OFF, val.length << 2);
     }
 
     /** {@inheritDoc} */
     @Override public void writeShort(int pos, short val) {
         ensureCapacity(pos + 2);
 
-        UNSAFE.putShort(data + pos, val);
+        GridUnsafe.putShort(data + pos, val);
     }
 
     /** {@inheritDoc} */
     @Override public void writeInt(int pos, int val) {
         ensureCapacity(pos + 4);
 
-        UNSAFE.putInt(data + pos, val);
+        GridUnsafe.putInt(data + pos, val);
     }
 
     /** {@inheritDoc} */
@@ -140,21 +132,21 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
     /** {@inheritDoc} */
     @Override public void writeFloatArray(float[] val) {
-        copyAndShift(val, FLOAT_ARR_OFF, val.length << 2);
+        copyAndShift(val, GridUnsafe.FLOAT_ARR_OFF, val.length << 2);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLong(long val) {
         ensureCapacity(pos + 8);
 
-        UNSAFE.putLong(data + pos, val);
+        GridUnsafe.putLong(data + pos, val);
 
         shift(8);
     }
 
     /** {@inheritDoc} */
     @Override public void writeLongArray(long[] val) {
-        copyAndShift(val, LONG_ARR_OFF, val.length << 3);
+        copyAndShift(val, GridUnsafe.LONG_ARR_OFF, val.length << 3);
     }
 
     /** {@inheritDoc} */
@@ -164,12 +156,12 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
     /** {@inheritDoc} */
     @Override public void writeDoubleArray(double[] val) {
-        copyAndShift(val, DOUBLE_ARR_OFF, val.length << 3);
+        copyAndShift(val, GridUnsafe.DOUBLE_ARR_OFF, val.length << 3);
     }
 
     /** {@inheritDoc} */
     @Override public void write(byte[] arr, int off, int len) {
-        copyAndShift(arr, BYTE_ARR_OFF + off, len);
+        copyAndShift(arr, GridUnsafe.BYTE_ARR_OFF + off, len);
     }
 
     /** {@inheritDoc} */
@@ -234,7 +226,7 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteByte(byte val) {
-        UNSAFE.putByte(data + pos++, val);
+        GridUnsafe.putByte(data + pos++, val);
     }
 
     /** {@inheritDoc} */
@@ -244,38 +236,38 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteShort(short val) {
-        UNSAFE.putShort(data + pos, val);
+        GridUnsafe.putShort(data + pos, val);
 
         shift(2);
     }
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteShort(int pos, short val) {
-        UNSAFE.putShort(data + pos, val);
+        GridUnsafe.putShort(data + pos, val);
     }
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteChar(char val) {
-        UNSAFE.putChar(data + pos, val);
+        GridUnsafe.putChar(data + pos, val);
 
         shift(2);
     }
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteInt(int val) {
-        UNSAFE.putInt(data + pos, val);
+        GridUnsafe.putInt(data + pos, val);
 
         shift(4);
     }
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteInt(int pos, int val) {
-        UNSAFE.putInt(data + pos, val);
+        GridUnsafe.putInt(data + pos, val);
     }
 
     /** {@inheritDoc} */
     @Override public void unsafeWriteLong(long val) {
-        UNSAFE.putLong(data + pos, val);
+        GridUnsafe.putLong(data + pos, val);
 
         shift(8);
     }
@@ -335,7 +327,7 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
     private void copyAndShift(Object src, long off, int len) {
         ensureCapacity(pos + len);
 
-        UNSAFE.copyMemory(src, off, null, data + pos, len);
+        GridUnsafe.copyMemory(src, off, null, data + pos, len);
 
         shift(len);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/util/GridHandleTable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridHandleTable.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridHandleTable.java
index 319a633..dd85bcf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridHandleTable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridHandleTable.java
@@ -18,19 +18,14 @@
 package org.apache.ignite.internal.util;
 
 import java.util.Arrays;
-import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.GridUnsafe.INT_ARR_OFF;
 
 /**
  * Lightweight identity hash table which maps objects to integer handles,
  * assigned in ascending order.
  */
 public class GridHandleTable {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private static final long intArrOff = UNSAFE.arrayBaseOffset(int[].class);
-
     /** Number of mappings in table/next available handle. */
     private int size;
 
@@ -111,8 +106,8 @@ public class GridHandleTable {
      * Resets table to its initial (empty) state.
      */
     public void clear() {
-        UNSAFE.copyMemory(spineEmpty, intArrOff, spine, intArrOff, spineEmpty.length << 2);
-        UNSAFE.copyMemory(nextEmpty, intArrOff, next, intArrOff, nextEmpty.length << 2);
+        GridUnsafe.copyMemory(spineEmpty, INT_ARR_OFF, spine, INT_ARR_OFF, spineEmpty.length << 2);
+        GridUnsafe.copyMemory(nextEmpty, INT_ARR_OFF, next, INT_ARR_OFF, nextEmpty.length << 2);
 
         Arrays.fill(objs, null);
 
@@ -153,7 +148,7 @@ public class GridHandleTable {
 
         Arrays.fill(spineEmpty, -1);
 
-        UNSAFE.copyMemory(spineEmpty, intArrOff, spine, intArrOff, spineEmpty.length << 2);
+        GridUnsafe.copyMemory(spineEmpty, INT_ARR_OFF, spine, INT_ARR_OFF, spineEmpty.length << 2);
 
         for (int i = 0; i < this.size; i++) {
             Object obj = objs[i];
@@ -171,7 +166,7 @@ public class GridHandleTable {
         int newLen = (next.length << 1) + 1;
         int[] newNext = new int[newLen];
 
-        UNSAFE.copyMemory(next, intArrOff, newNext, intArrOff, size << 2);
+        GridUnsafe.copyMemory(next, INT_ARR_OFF, newNext, INT_ARR_OFF, size << 2);
 
         next = newNext;
         nextEmpty = new int[newLen];

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index 3371eb8..8a0b0ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -128,9 +128,6 @@ public final class GridJavaProcess {
     public static GridJavaProcess exec(String clsName, String params, @Nullable IgniteLogger log,
         @Nullable IgniteInClosure<String> printC, @Nullable GridAbsClosure procKilledC,
         @Nullable String javaHome, @Nullable Collection<String> jvmArgs, @Nullable String cp) throws Exception {
-        if (!(U.isLinux() || U.isMacOs() || U.isWindows()))
-            throw new Exception("Your OS is not supported.");
-
         GridJavaProcess gjProc = new GridJavaProcess();
 
         gjProc.log = log;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a87decdc/modules/core/src/main/java/org/apache/ignite/internal/util/GridSpinReadWriteLock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridSpinReadWriteLock.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridSpinReadWriteLock.java
index a1fa892..4f23979 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridSpinReadWriteLock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridSpinReadWriteLock.java
@@ -21,7 +21,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import sun.misc.Unsafe;
 
 /**
  *
@@ -29,9 +28,6 @@ import sun.misc.Unsafe;
 @GridToStringExclude
 public class GridSpinReadWriteLock {
     /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
     private static final long PENDING_WLOCKS_OFFS;
 
     /** */
@@ -42,10 +38,10 @@ public class GridSpinReadWriteLock {
      */
     static {
         try {
-            STATE_OFFS = UNSAFE.objectFieldOffset(GridSpinReadWriteLock.class.getDeclaredField("state"));
+            STATE_OFFS = GridUnsafe.objectFieldOffset(GridSpinReadWriteLock.class.getDeclaredField("state"));
 
             PENDING_WLOCKS_OFFS =
-                UNSAFE.objectFieldOffset(GridSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
+                GridUnsafe.objectFieldOffset(GridSpinReadWriteLock.class.getDeclaredField("pendingWLocks"));
         }
         catch (NoSuchFieldException e) {
             throw new Error(e);
@@ -403,7 +399,7 @@ public class GridSpinReadWriteLock {
      * @return {@code True} on success.
      */
     private boolean compareAndSet(long offs, int expect, int update) {
-        return UNSAFE.compareAndSwapInt(this, offs, expect, update);
+        return GridUnsafe.compareAndSwapInt(this, offs, expect, update);
     }
 
     /** {@inheritDoc} */