You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/04/02 15:57:54 UTC

[GitHub] [ignite-3] SammyVimes opened a new pull request #82: IGNITE-14084 Ignite direct message writer and reader

SammyVimes opened a new pull request #82:
URL: https://github.com/apache/ignite-3/pull/82


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] agoncharuk commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r608659731



##########
File path: modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageCollectionItemType.java
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.plugin.extensions.communication;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Enum representing possible types of collection items.
+ */
+public enum MessageCollectionItemType {
+    /** Byte. */
+    BYTE,
+
+    /** Short. */
+    SHORT,
+
+    /** Integer. */
+    INT,
+
+    /** Long. */
+    LONG,
+
+    /** Float. */
+    FLOAT,
+
+    /** Double. */
+    DOUBLE,
+
+    /** Character. */
+    CHAR,
+
+    /** Boolean. */
+    BOOLEAN,
+
+    /** Byte array. */
+    BYTE_ARR,
+
+    /** Short array. */
+    SHORT_ARR,
+
+    /** Integer array. */
+    INT_ARR,
+
+    /** Long array. */
+    LONG_ARR,
+
+    /** Float array. */
+    FLOAT_ARR,
+
+    /** Double array. */
+    DOUBLE_ARR,
+
+    /** Character array. */
+    CHAR_ARR,
+
+    /** Boolean array. */
+    BOOLEAN_ARR,
+
+    /** String. */
+    STRING,
+
+    /** Bit set. */
+    BIT_SET,
+
+    /** UUID. */
+    UUID,
+
+    /** Ignite UUID. */
+    IGNITE_UUID,
+
+    /** Message. */
+    MSG,
+
+    /** Topology version. */
+    AFFINITY_TOPOLOGY_VERSION;

Review comment:
       This likely can be removed - looks like a compatibility artifact in Ignite 2.

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/direct/DirectMessageWriter.java
##########
@@ -0,0 +1,373 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.network.internal.direct;

Review comment:
       Why only message readers are placed in .internal package? Let's move the whole module in org.apache.ignite.internal.network package?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sashapolo commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609570012



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/DirectBufferCleaner.java
##########
@@ -15,24 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.network.scalecube;
+package org.apache.ignite.internal.util;
 
-import java.io.ObjectOutputStream;
-import org.apache.ignite.network.internal.MessageWriter;
+import java.nio.ByteBuffer;
 
-/** */
-@Deprecated
-public class ScaleCubeMessageWriter implements MessageWriter {
-    /** */
-    private final ObjectOutputStream stream;
-
-    /** */
-    public ScaleCubeMessageWriter(ObjectOutputStream stream) {
-        this.stream = stream;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ObjectOutputStream stream() {
-        return stream;
-    }
+/**
+ * Cleaner interface for {@code java.nio.ByteBuffer}.
+ */
+public interface DirectBufferCleaner {
+    /**
+     * Cleans direct buffer.

Review comment:
       Can you clarify what does "cleaning" the buffer mean?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:

Review comment:
       ```suggestion
    * All memory access operations have the following properties:
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/FeatureChecker.java
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util;
+
+/**
+ * Class extracted for fields from GridUnsafe to be absolutely independent with current and future static block
+ * initialization effects.
+ */
+public class FeatureChecker {
+    /** Required Options to Run on Java 9, 10, 11. */

Review comment:
       ```suggestion
       /** Required options to run on Java 9, 10, 11. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.

Review comment:
       ```suggestion
        * Wraps a pointer to unmanaged memory into a direct byte buffer.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageReader.java
##########
@@ -17,13 +17,292 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message reader.
  */
 public interface MessageReader {
-    /** Get input stream. Will be replaced with Ignite 2.X message reader interface. */
-    @Deprecated
-    ObjectInputStream stream();
+    /**
+     * Sets a byte buffer to read from.
+     *
+     * @param buf Byte buffer.
+     */
+    public void setBuffer(ByteBuffer buf);
+
+    /**
+     * Sets the type of a message that is being currently read.
+     *
+     * @param msgCls Message type.
+     */
+    public void setCurrentReadClass(Class<? extends NetworkMessage> msgCls);
+
+    /**
+     * Callback that must be invoked by the implementation of a message serializer before message body started decoding.

Review comment:
       ```suggestion
        * Callback that must be invoked by implementations of message serializers before they start decoding the message body.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageReader.java
##########
@@ -17,13 +17,292 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message reader.
  */
 public interface MessageReader {
-    /** Get input stream. Will be replaced with Ignite 2.X message reader interface. */
-    @Deprecated
-    ObjectInputStream stream();
+    /**
+     * Sets a byte buffer to read from.
+     *
+     * @param buf Byte buffer.
+     */
+    public void setBuffer(ByteBuffer buf);
+
+    /**
+     * Sets the type of a message that is being currently read.
+     *
+     * @param msgCls Message type.
+     */
+    public void setCurrentReadClass(Class<? extends NetworkMessage> msgCls);
+
+    /**
+     * Callback that must be invoked by the implementation of a message serializer before message body started decoding.
+     *
+     * @return {@code True} if a read operation can be proceeded, {@code false} otherwise.
+     */
+    public boolean beforeMessageRead();
+
+    /**
+     * Callback that must be invoked by the implementation of a message serializer after message body finished decoding.

Review comment:
       ```suggestion
        * Callback that must be invoked by implementations of message serializers after they finished decoding the message body.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/message/MessageSerializer.java
##########
@@ -25,10 +25,11 @@
  */
 public interface MessageSerializer<M extends NetworkMessage> {
     /**
-     * Write message to writer.
+     * Write a message to writer.

Review comment:
       ```suggestion
        * Writes a message to the writer.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.

Review comment:
       ```suggestion
        * Wraps a pointer to unmanaged memory with a direct byte buffer using the direct byte buffer's constructor.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>

Review comment:
       ```suggestion
    * invoking these methods.</li>
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.

Review comment:
       ```suggestion
        * Wraps a pointer to unmanaged memory with a direct byte buffer using a JavaNioAccess object.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param newDirectBufMtd Method which should return an instance of a direct byte buffer.
+     * @param javaNioAccessObj Object to invoke method.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerJavaNio(long ptr,
+                                                          int len,
+                                                          @NotNull Method newDirectBufMtd,
+                                                          @NotNull Object javaNioAccessObj) {
+        try {
+            ByteBuffer buf = (ByteBuffer)newDirectBufMtd.invoke(javaNioAccessObj, ptr, len, null);
+
+            assert buf.isDirect();
+
+            buf.order(NATIVE_BYTE_ORDER);
+
+            return buf;
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("JavaNioAccess#newDirectByteBuffer() method is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * @param len Length.
+     * @return Allocated direct buffer.
+     */
+    public static ByteBuffer allocateBuffer(int len) {
+        long ptr = allocateMemory(len);
+
+        return wrapPointer(ptr, len);
+    }
+
+    /**
+     * @param buf Direct buffer allocated by {@link #allocateBuffer(int)}.
+     */
+    public static void freeBuffer(ByteBuffer buf) {
+        long ptr = bufferAddress(buf);
+
+        freeMemory(ptr);
+    }
+
+    /**
+     *
+     * @param buf Buffer.
+     * @param len New length.
+     * @return Reallocated direct buffer.
+     */
+    public static ByteBuffer reallocateBuffer(ByteBuffer buf, int len) {
+        long ptr = bufferAddress(buf);
+
+        long newPtr = reallocateMemory(ptr, len);
+
+        return wrapPointer(newPtr, len);
+    }
+
+    /**
+     * Gets boolean value from object field.

Review comment:
       ```suggestion
        * Gets a boolean value from an object field.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param newDirectBufMtd Method which should return an instance of a direct byte buffer.
+     * @param javaNioAccessObj Object to invoke method.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerJavaNio(long ptr,
+                                                          int len,
+                                                          @NotNull Method newDirectBufMtd,
+                                                          @NotNull Object javaNioAccessObj) {
+        try {
+            ByteBuffer buf = (ByteBuffer)newDirectBufMtd.invoke(javaNioAccessObj, ptr, len, null);
+
+            assert buf.isDirect();
+
+            buf.order(NATIVE_BYTE_ORDER);
+
+            return buf;
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("JavaNioAccess#newDirectByteBuffer() method is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * @param len Length.
+     * @return Allocated direct buffer.
+     */
+    public static ByteBuffer allocateBuffer(int len) {
+        long ptr = allocateMemory(len);
+
+        return wrapPointer(ptr, len);
+    }
+
+    /**
+     * @param buf Direct buffer allocated by {@link #allocateBuffer(int)}.
+     */
+    public static void freeBuffer(ByteBuffer buf) {
+        long ptr = bufferAddress(buf);
+
+        freeMemory(ptr);
+    }
+
+    /**
+     *
+     * @param buf Buffer.
+     * @param len New length.
+     * @return Reallocated direct buffer.
+     */
+    public static ByteBuffer reallocateBuffer(ByteBuffer buf, int len) {
+        long ptr = bufferAddress(buf);
+
+        long newPtr = reallocateMemory(ptr, len);
+
+        return wrapPointer(newPtr, len);
+    }
+
+    /**
+     * Gets boolean value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Boolean value from object field.
+     */
+    public static boolean getBooleanField(Object obj, long fieldOff) {
+        return UNSAFE.getBoolean(obj, fieldOff);
+    }
+
+    /**
+     * Stores boolean value into object field.

Review comment:
       ```suggestion
        * Stores a boolean value into an object field.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param newDirectBufMtd Method which should return an instance of a direct byte buffer.
+     * @param javaNioAccessObj Object to invoke method.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerJavaNio(long ptr,
+                                                          int len,
+                                                          @NotNull Method newDirectBufMtd,
+                                                          @NotNull Object javaNioAccessObj) {
+        try {
+            ByteBuffer buf = (ByteBuffer)newDirectBufMtd.invoke(javaNioAccessObj, ptr, len, null);
+
+            assert buf.isDirect();
+
+            buf.order(NATIVE_BYTE_ORDER);
+
+            return buf;
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("JavaNioAccess#newDirectByteBuffer() method is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * @param len Length.
+     * @return Allocated direct buffer.
+     */
+    public static ByteBuffer allocateBuffer(int len) {
+        long ptr = allocateMemory(len);
+
+        return wrapPointer(ptr, len);
+    }
+
+    /**
+     * @param buf Direct buffer allocated by {@link #allocateBuffer(int)}.
+     */
+    public static void freeBuffer(ByteBuffer buf) {
+        long ptr = bufferAddress(buf);
+
+        freeMemory(ptr);
+    }
+
+    /**
+     *
+     * @param buf Buffer.
+     * @param len New length.
+     * @return Reallocated direct buffer.
+     */
+    public static ByteBuffer reallocateBuffer(ByteBuffer buf, int len) {
+        long ptr = bufferAddress(buf);
+
+        long newPtr = reallocateMemory(ptr, len);
+
+        return wrapPointer(newPtr, len);
+    }
+
+    /**
+     * Gets boolean value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Boolean value from object field.
+     */
+    public static boolean getBooleanField(Object obj, long fieldOff) {
+        return UNSAFE.getBoolean(obj, fieldOff);
+    }
+
+    /**
+     * Stores boolean value into object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @param val Value.
+     */
+    public static void putBooleanField(Object obj, long fieldOff, boolean val) {
+        UNSAFE.putBoolean(obj, fieldOff, val);
+    }
+
+    /**
+     * Gets byte value from object field.

Review comment:
       ```suggestion
        * Gets a byte value from an object field.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageReader.java
##########
@@ -17,13 +17,292 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message reader.
  */
 public interface MessageReader {
-    /** Get input stream. Will be replaced with Ignite 2.X message reader interface. */
-    @Deprecated
-    ObjectInputStream stream();
+    /**
+     * Sets a byte buffer to read from.
+     *
+     * @param buf Byte buffer.
+     */
+    public void setBuffer(ByteBuffer buf);
+
+    /**
+     * Sets the type of a message that is being currently read.

Review comment:
       ```suggestion
        * Sets the type of the message that is currently being read.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param newDirectBufMtd Method which should return an instance of a direct byte buffer.
+     * @param javaNioAccessObj Object to invoke method.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerJavaNio(long ptr,
+                                                          int len,
+                                                          @NotNull Method newDirectBufMtd,
+                                                          @NotNull Object javaNioAccessObj) {
+        try {
+            ByteBuffer buf = (ByteBuffer)newDirectBufMtd.invoke(javaNioAccessObj, ptr, len, null);
+
+            assert buf.isDirect();
+
+            buf.order(NATIVE_BYTE_ORDER);
+
+            return buf;
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("JavaNioAccess#newDirectByteBuffer() method is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * @param len Length.
+     * @return Allocated direct buffer.
+     */
+    public static ByteBuffer allocateBuffer(int len) {
+        long ptr = allocateMemory(len);
+
+        return wrapPointer(ptr, len);
+    }
+
+    /**
+     * @param buf Direct buffer allocated by {@link #allocateBuffer(int)}.
+     */
+    public static void freeBuffer(ByteBuffer buf) {
+        long ptr = bufferAddress(buf);
+
+        freeMemory(ptr);
+    }
+
+    /**
+     *
+     * @param buf Buffer.
+     * @param len New length.
+     * @return Reallocated direct buffer.
+     */
+    public static ByteBuffer reallocateBuffer(ByteBuffer buf, int len) {
+        long ptr = bufferAddress(buf);
+
+        long newPtr = reallocateMemory(ptr, len);
+
+        return wrapPointer(newPtr, len);
+    }
+
+    /**
+     * Gets boolean value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Boolean value from object field.
+     */
+    public static boolean getBooleanField(Object obj, long fieldOff) {
+        return UNSAFE.getBoolean(obj, fieldOff);
+    }
+
+    /**
+     * Stores boolean value into object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @param val Value.
+     */
+    public static void putBooleanField(Object obj, long fieldOff, boolean val) {
+        UNSAFE.putBoolean(obj, fieldOff, val);
+    }
+
+    /**
+     * Gets byte value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Byte value from object field.
+     */
+    public static byte getByteField(Object obj, long fieldOff) {
+        return UNSAFE.getByte(obj, fieldOff);
+    }
+
+    /**
+     * Stores byte value into object field.

Review comment:
       same stuff

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for the {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that the byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufferConstructor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufferConstructor(long ptr, int len, Constructor<?> constructor) {
+        try {
+            Object newDirectBuf = constructor.newInstance(ptr, len);
+
+            return ((ByteBuffer)newDirectBuf).order(NATIVE_BYTE_ORDER);
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("DirectByteBuffer#constructor is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses JavaNioAccess object.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param newDirectBufMtd Method which should return an instance of a direct byte buffer.
+     * @param javaNioAccessObj Object to invoke method.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerJavaNio(long ptr,
+                                                          int len,
+                                                          @NotNull Method newDirectBufMtd,
+                                                          @NotNull Object javaNioAccessObj) {
+        try {
+            ByteBuffer buf = (ByteBuffer)newDirectBufMtd.invoke(javaNioAccessObj, ptr, len, null);
+
+            assert buf.isDirect();
+
+            buf.order(NATIVE_BYTE_ORDER);
+
+            return buf;
+        }
+        catch (ReflectiveOperationException e) {
+            throw new RuntimeException("JavaNioAccess#newDirectByteBuffer() method is unavailable."
+                + FeatureChecker.JAVA_VER_SPECIFIC_WARN, e);
+        }
+    }
+
+    /**
+     * @param len Length.
+     * @return Allocated direct buffer.
+     */
+    public static ByteBuffer allocateBuffer(int len) {
+        long ptr = allocateMemory(len);
+
+        return wrapPointer(ptr, len);
+    }
+
+    /**
+     * @param buf Direct buffer allocated by {@link #allocateBuffer(int)}.
+     */
+    public static void freeBuffer(ByteBuffer buf) {
+        long ptr = bufferAddress(buf);
+
+        freeMemory(ptr);
+    }
+
+    /**
+     *
+     * @param buf Buffer.
+     * @param len New length.
+     * @return Reallocated direct buffer.
+     */
+    public static ByteBuffer reallocateBuffer(ByteBuffer buf, int len) {
+        long ptr = bufferAddress(buf);
+
+        long newPtr = reallocateMemory(ptr, len);
+
+        return wrapPointer(newPtr, len);
+    }
+
+    /**
+     * Gets boolean value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Boolean value from object field.
+     */
+    public static boolean getBooleanField(Object obj, long fieldOff) {
+        return UNSAFE.getBoolean(obj, fieldOff);
+    }
+
+    /**
+     * Stores boolean value into object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @param val Value.
+     */
+    public static void putBooleanField(Object obj, long fieldOff, boolean val) {
+        UNSAFE.putBoolean(obj, fieldOff, val);
+    }
+
+    /**
+     * Gets byte value from object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @return Byte value from object field.
+     */
+    public static byte getByteField(Object obj, long fieldOff) {
+        return UNSAFE.getByte(obj, fieldOff);
+    }
+
+    /**
+     * Stores byte value into object field.
+     *
+     * @param obj Object.
+     * @param fieldOff Field offset.
+     * @param val Value.
+     */
+    public static void putByteField(Object obj, long fieldOff, byte val) {
+        UNSAFE.putByte(obj, fieldOff, val);
+    }
+
+    /**
+     * Gets short value from object field.

Review comment:
       same

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageSerializerFactory.java
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.network.internal;
+
+import java.util.List;
+import org.apache.ignite.network.message.MessageDeserializer;
+import org.apache.ignite.network.message.MessageSerializer;
+import org.apache.ignite.network.message.MessageSerializerProvider;
+import org.apache.ignite.network.message.NetworkMessage;
+
+/**
+ * Factory that provides message serializers and deserializers by {@link NetworkMessage#directType()}.
+ */
+public class MessageSerializerFactory {
+    /** List of all serializers. Index is the direct type of the message. */
+    private final List<MessageSerializerProvider<NetworkMessage>> serializerProviders;
+
+    /** Constructor. */
+    public MessageSerializerFactory(List<MessageSerializerProvider<NetworkMessage>> mappers) {
+        serializerProviders = mappers;
+    }
+
+    /**
+     * Create a deserializer for a message by the direct type.

Review comment:
       ```suggestion
        * Creates a deserializer for a message of the given direct type.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageReader.java
##########
@@ -17,13 +17,292 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message reader.
  */
 public interface MessageReader {
-    /** Get input stream. Will be replaced with Ignite 2.X message reader interface. */
-    @Deprecated
-    ObjectInputStream stream();
+    /**
+     * Sets a byte buffer to read from.

Review comment:
       ```suggestion
        * Sets the byte buffer to read from.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageReader.java
##########
@@ -17,13 +17,292 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message reader.
  */
 public interface MessageReader {
-    /** Get input stream. Will be replaced with Ignite 2.X message reader interface. */
-    @Deprecated
-    ObjectInputStream stream();
+    /**
+     * Sets a byte buffer to read from.
+     *
+     * @param buf Byte buffer.
+     */
+    public void setBuffer(ByteBuffer buf);
+
+    /**
+     * Sets the type of a message that is being currently read.
+     *
+     * @param msgCls Message type.
+     */
+    public void setCurrentReadClass(Class<? extends NetworkMessage> msgCls);
+
+    /**
+     * Callback that must be invoked by the implementation of a message serializer before message body started decoding.
+     *
+     * @return {@code True} if a read operation can be proceeded, {@code false} otherwise.

Review comment:
       ```suggestion
        * @return {@code True} if a read operation is allowed to proceed, {@code false} otherwise.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageSerializerFactory.java
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.network.internal;
+
+import java.util.List;
+import org.apache.ignite.network.message.MessageDeserializer;
+import org.apache.ignite.network.message.MessageSerializer;
+import org.apache.ignite.network.message.MessageSerializerProvider;
+import org.apache.ignite.network.message.NetworkMessage;
+
+/**
+ * Factory that provides message serializers and deserializers by {@link NetworkMessage#directType()}.
+ */
+public class MessageSerializerFactory {
+    /** List of all serializers. Index is the direct type of the message. */
+    private final List<MessageSerializerProvider<NetworkMessage>> serializerProviders;
+
+    /** Constructor. */
+    public MessageSerializerFactory(List<MessageSerializerProvider<NetworkMessage>> mappers) {
+        serializerProviders = mappers;
+    }
+
+    /**
+     * Create a deserializer for a message by the direct type.
+     * @param directType Message's direct type.
+     * @return Message deserializer.
+     */
+    public MessageDeserializer<NetworkMessage> createDeserializer(short directType) {
+        return serializerProviders.get(directType).createDeserializer();
+    }
+
+    /**
+     * Create a serializer for a message by the direct type.

Review comment:
       ```suggestion
        * Creates a serializer for a message of the given direct type.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageWriter.java
##########
@@ -17,13 +17,320 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message writer.
  */
 public interface MessageWriter {
-    /** Get output stream. Will be replaced with Ignite 2.X message writer interface. */
-    @Deprecated
-    ObjectOutputStream stream();
+    /**
+     * Sets a byte buffer to write to.

Review comment:
       ```suggestion
        * Sets the byte buffer to write to.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/direct/state/DirectMessageStateItem.java
##########
@@ -15,24 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.network.scalecube;
+package org.apache.ignite.network.internal.direct.state;
 
-import java.io.ObjectInputStream;
-import org.apache.ignite.network.internal.MessageReader;
-
-/** */
-@Deprecated
-public class ScaleCubeMessageReader implements MessageReader {
-    /** */
-    private final ObjectInputStream stream;
-
-    /** */
-    public ScaleCubeMessageReader(ObjectInputStream stream) {
-        this.stream = stream;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ObjectInputStream stream() {
-        return this.stream;
-    }
+/**
+ * Message state item.
+ */
+public interface DirectMessageStateItem {
+    /**
+     * Resets state.

Review comment:
       ```suggestion
        * Resets the state.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/message/MessageDeserializer.java
##########
@@ -25,10 +25,14 @@
  */
 public interface MessageDeserializer<M extends NetworkMessage> {
     /**
-     * Read message from reader.
+     * Read a message from reader.
      * @param reader Message reader.
-     * @return Read message.
+     * @return {@code true } if the message was completely read.

Review comment:
       ```suggestion
        * @return {@code true } if the message has been read completely.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/message/MessageDeserializer.java
##########
@@ -25,10 +25,14 @@
  */
 public interface MessageDeserializer<M extends NetworkMessage> {
     /**
-     * Read message from reader.
+     * Read a message from reader.

Review comment:
       ```suggestion
        * Read a message from the reader.
   ```

##########
File path: modules/network/src/main/java/org/apache/ignite/network/internal/MessageWriter.java
##########
@@ -17,13 +17,320 @@
 
 package org.apache.ignite.network.internal;
 
-import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 
 /**
  * Stateful message writer.
  */
 public interface MessageWriter {
-    /** Get output stream. Will be replaced with Ignite 2.X message writer interface. */
-    @Deprecated
-    ObjectOutputStream stream();
+    /**
+     * Sets a byte buffer to write to.
+     *
+     * @param buf Byte buffer.
+     */
+    public void setBuffer(ByteBuffer buf);
+
+    /**
+     * Sets the type of a message that is being currently written.

Review comment:
       ```suggestion
        * Sets the type of the message that is currently being written.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/DirectBufferCleaner.java
##########
@@ -15,24 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.network.scalecube;
+package org.apache.ignite.internal.util;
 
-import java.io.ObjectOutputStream;
-import org.apache.ignite.network.internal.MessageWriter;
+import java.nio.ByteBuffer;
 
-/** */
-@Deprecated
-public class ScaleCubeMessageWriter implements MessageWriter {
-    /** */
-    private final ObjectOutputStream stream;
-
-    /** */
-    public ScaleCubeMessageWriter(ObjectOutputStream stream) {
-        this.stream = stream;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ObjectOutputStream stream() {
-        return stream;
-    }
+/**
+ * Cleaner interface for {@code java.nio.ByteBuffer}.
+ */
+public interface DirectBufferCleaner {
+    /**
+     * Cleans direct buffer.

Review comment:
       ```suggestion
        * Cleans the direct buffer.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sashapolo commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609870656



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/DirectBufferCleaner.java
##########
@@ -15,24 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.network.scalecube;
+package org.apache.ignite.internal.util;
 
-import java.io.ObjectOutputStream;
-import org.apache.ignite.network.internal.MessageWriter;
+import java.nio.ByteBuffer;
 
-/** */
-@Deprecated
-public class ScaleCubeMessageWriter implements MessageWriter {
-    /** */
-    private final ObjectOutputStream stream;
-
-    /** */
-    public ScaleCubeMessageWriter(ObjectOutputStream stream) {
-        this.stream = stream;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ObjectOutputStream stream() {
-        return stream;
-    }
+/**
+ * Cleaner interface for {@code java.nio.ByteBuffer}.
+ */
+public interface DirectBufferCleaner {
+    /**
+     * Cleans direct buffer.

Review comment:
       Can you clarify that in the javadoc?) Or is it considered some well-known terminology?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r608596929



##########
File path: modules/core/src/main/java/org/apache/ignite/IgniteException.java
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * General grid exception. This exception is used to indicate any error condition
+ * within Grid.
+ */
+public class IgniteException extends RuntimeException {

Review comment:
       Can we reuse an exception added in ignite-14495 (pr-85, already in main) instead of this one? I think this is internal exception that should not be propagated to the end user.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ascherbakoff commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
ascherbakoff commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609717385



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();

Review comment:
       Can you clarify the purpose ? Just curious.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] sergey-chugunov-1985 commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
sergey-chugunov-1985 commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r608597500



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/ReflectiveDirectBufferCleaner.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+
+/**
+ * {@link DirectBufferCleaner} implementation based on {@code sun.misc.Cleaner} and
+ * {@code sun.nio.ch.DirectBuffer.cleaner()} method.
+ *
+ * Mote: This implementation will not work on Java 9+.

Review comment:
       Typo: `Note`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] alievmirza commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
alievmirza commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r608614511



##########
File path: modules/network/src/integrationTest/java/org/apache/ignite/network/DirectSerializationTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.network;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.network.internal.MessageSerializerFactory;
+import org.apache.ignite.network.internal.direct.DirectMessageReader;
+import org.apache.ignite.network.internal.direct.DirectMessageWriter;
+import org.apache.ignite.network.message.MessageDeserializer;
+import org.apache.ignite.network.message.MessageSerializer;
+import org.apache.ignite.network.message.MessageSerializerProvider;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.network.scalecube.TestMessage;
+import org.apache.ignite.network.scalecube.TestMessageSerializerProvider;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test Direct Message Writing/Reading works. This test won't be needed after we implement Netty Transport for Ignite.

Review comment:
       If there is a ticket for Netty Transport, it would be nice to mention it 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] ascherbakoff commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
ascherbakoff commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609534971



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufCtor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternative for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);

Review comment:
       ```suggestion
               throw new RuntimeException("All alternatives for a new DirectByteBuffer() creation have failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();

Review comment:
       printStackTrace should be avoided

##########
File path: modules/network/src/integrationTest/java/org/apache/ignite/network/DirectSerializationTest.java
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.network;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.network.internal.MessageSerializerFactory;
+import org.apache.ignite.network.internal.direct.DirectMessageReader;
+import org.apache.ignite.network.internal.direct.DirectMessageWriter;
+import org.apache.ignite.network.message.MessageDeserializer;
+import org.apache.ignite.network.message.MessageSerializer;
+import org.apache.ignite.network.message.MessageSerializerProvider;
+import org.apache.ignite.network.message.NetworkMessage;
+import org.apache.ignite.network.scalecube.TestMessage;
+import org.apache.ignite.network.scalecube.TestMessageSerializerProvider;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test Direct Message Writing/Reading works. This test won't be needed after we implement Netty Transport
+ * for Ignite (IGNITE-14088).
+ */
+public class DirectSerializationTest {

Review comment:
       The test message should contain all possible field types. I'm ok if it will be done as a separate ticket.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; // fallback was not suceefull
+                }
+
+                if (directBufCtor == null)
+                    throw e;
+            }
+        }
+        else {
+            try {
+                directBufCtor = createAndTestNewDirectBufferCtor();
+            }
+            catch (Exception e) {
+                try {
+                    nioAccessObj = javaNioAccessObject();
+                    directBufMtd = newDirectBufferMethod(nioAccessObj);
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();
+
+                    e.addSuppressed(eFallback);
+
+                    throw e; //fallback to shared secrets failed.
+                }
+
+                if (nioAccessObj == null || directBufMtd == null)
+                    throw e;
+            }
+        }
+
+        JAVA_NIO_ACCESS_OBJ = nioAccessObj;
+        NEW_DIRECT_BUF_MTD = directBufMtd;
+
+        NEW_DIRECT_BUF_CONSTRUCTOR = directBufCtor;
+    }
+
+    /**
+     * Ensure singleton.
+     */
+    private GridUnsafe() {
+        // No-op.
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @return Byte buffer wrapping the given memory.
+     */
+    public static ByteBuffer wrapPointer(long ptr, int len) {
+        if (NEW_DIRECT_BUF_MTD != null && JAVA_NIO_ACCESS_OBJ != null)
+            return wrapPointerJavaNio(ptr, len, NEW_DIRECT_BUF_MTD, JAVA_NIO_ACCESS_OBJ);
+        else if (NEW_DIRECT_BUF_CONSTRUCTOR != null)
+            return wrapPointerDirectBufCtor(ptr, len, NEW_DIRECT_BUF_CONSTRUCTOR);
+        else
+            throw new RuntimeException("All alternative for a new DirectByteBuffer() creation failed: " + FeatureChecker.JAVA_VER_SPECIFIC_WARN);
+    }
+
+    /**
+     * Wraps pointer to unmanaged memory into direct byte buffer. Uses constructor of a direct byte buffer.
+     *
+     * @param ptr Pointer to wrap.
+     * @param len Memory location length.
+     * @param constructor Constructor to use. Should create an instance of a direct ByteBuffer.
+     * @return Byte buffer wrapping the given memory.
+     */
+    @NotNull private static ByteBuffer wrapPointerDirectBufCtor(long ptr, int len, Constructor<?> constructor) {

Review comment:
       Abbreviations in method names are not allowed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609573063



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();

Review comment:
       @agura and @ibessonov verified that printStackTrace is here on purpose. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609849536



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/DirectBufferCleaner.java
##########
@@ -15,24 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.network.scalecube;
+package org.apache.ignite.internal.util;
 
-import java.io.ObjectOutputStream;
-import org.apache.ignite.network.internal.MessageWriter;
+import java.nio.ByteBuffer;
 
-/** */
-@Deprecated
-public class ScaleCubeMessageWriter implements MessageWriter {
-    /** */
-    private final ObjectOutputStream stream;
-
-    /** */
-    public ScaleCubeMessageWriter(ObjectOutputStream stream) {
-        this.stream = stream;
-    }
-
-    /** {@inheritDoc} */
-    @Override public ObjectOutputStream stream() {
-        return stream;
-    }
+/**
+ * Cleaner interface for {@code java.nio.ByteBuffer}.
+ */
+public interface DirectBufferCleaner {
+    /**
+     * Cleans direct buffer.

Review comment:
       Freeing the memory




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes commented on a change in pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
SammyVimes commented on a change in pull request #82:
URL: https://github.com/apache/ignite-3/pull/82#discussion_r609847453



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
##########
@@ -0,0 +1,2014 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static org.apache.ignite.internal.util.IgniteUtils.jdkVersion;
+import static org.apache.ignite.internal.util.IgniteUtils.majorJavaVersion;
+
+/**
+ * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
+ *
+ * <p>
+ * The following statements for memory access operations  are true:
+ * <ul>
+ * <li>All {@code putXxx(long addr, xxx val)}, {@code getXxx(long addr)}, {@code putXxx(byte[] arr, long off, xxx val)},
+ * {@code getXxx(byte[] arr, long off)} and corresponding methods with {@code LE} suffix are alignment aware
+ * and can be safely used with unaligned pointers.</li>
+ * <li>All {@code putXxxField(Object obj, long fieldOff, xxx val)} and {@code getXxxField(Object obj, long fieldOff)}
+ * methods are not alignment aware and can't be safely used with unaligned pointers. This methods can be safely used
+ * for object field values access because all object fields addresses are aligned.</li>
+ * <li>All {@code putXxxLE(...)} and {@code getXxxLE(...)} methods assumes that byte order is fixed as little-endian
+ * while native byte order is big-endian. So it is client code responsibility to check native byte order before
+ * invoking of this methods.</li>
+ * </ul>
+ * </p>
+ */
+public abstract class GridUnsafe {
+    /** */
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
+    /** Unsafe. */
+    private static final Unsafe UNSAFE = unsafe();
+
+    /** Unaligned flag. */
+    private static final boolean UNALIGNED = unaligned();
+
+    /** Per-byte copy threshold. */
+    private static final long PER_BYTE_THRESHOLD = 0L;
+
+    /** Big endian. */
+    public static final boolean BIG_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN;
+
+    /** Address size. */
+    public static final int ADDR_SIZE = UNSAFE.addressSize();
+
+    /** */
+    public static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final int BYTE_ARR_INT_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    public static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
+
+    /** */
+    public static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
+
+    /** */
+    public static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
+
+    /** */
+    public static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
+
+    /** */
+    public static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
+
+    /** */
+    public static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
+
+    /** */
+    public static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
+
+    /** {@link java.nio.Buffer#address} field offset. */
+    private static final long DIRECT_BUF_ADDR_OFF = bufferAddressOffset();
+
+    /** Cleaner code for direct {@code java.nio.ByteBuffer}. */
+    private static final DirectBufferCleaner DIRECT_BUF_CLEANER =
+        majorJavaVersion(jdkVersion()) < 9
+            ? new ReflectiveDirectBufferCleaner()
+            : new UnsafeDirectBufferCleaner();
+
+    /** JavaNioAccess object. If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available. */
+    @Nullable private static final Object JAVA_NIO_ACCESS_OBJ;
+
+    /**
+     * JavaNioAccess#newDirectByteBuffer method. Ususally {@code null} if {@link #JAVA_NIO_ACCESS_OBJ} is {@code null}.
+     * If {@code null} then {@link #NEW_DIRECT_BUF_CONSTRUCTOR} should be available.
+     */
+    @Nullable private static final Method NEW_DIRECT_BUF_MTD;
+
+    /**
+     * New direct buffer class constructor obtained and tested using reflection. If {@code null} then both {@link
+     * #JAVA_NIO_ACCESS_OBJ} and {@link #NEW_DIRECT_BUF_MTD} should be not {@code null}.
+     */
+    @Nullable private static final Constructor<?> NEW_DIRECT_BUF_CONSTRUCTOR;
+
+    static {
+        Object nioAccessObj = null;
+        Method directBufMtd = null;
+
+        Constructor<?> directBufCtor = null;
+
+        if (majorJavaVersion(jdkVersion()) < 12) {
+            // for old java prefer Java NIO & Shared Secrets obect init way
+            try {
+                nioAccessObj = javaNioAccessObject();
+                directBufMtd = newDirectBufferMethod(nioAccessObj);
+            }
+            catch (Exception e) {
+                nioAccessObj = null;
+                directBufMtd = null;
+
+                try {
+                    directBufCtor = createAndTestNewDirectBufferCtor();
+                }
+                catch (Exception eFallback) {
+                    //noinspection CallToPrintStackTrace
+                    eFallback.printStackTrace();

Review comment:
       Stacktrace is being printed just before the process crashes, so it's the last thing we can do before any information about crash is lost




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite-3] SammyVimes closed pull request #82: IGNITE-14084 Ignite direct message writer and reader

Posted by GitBox <gi...@apache.org>.
SammyVimes closed pull request #82:
URL: https://github.com/apache/ignite-3/pull/82


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org