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

[1/2] ignite git commit: Squashed commit of the following:

Repository: ignite
Updated Branches:
  refs/heads/ignite-2329 [created] 54780f8c5


Squashed commit of the following:

commit 4c5df41aedba5870043cce302e4bfb8aca62942a
Merge: afd3bc1 f095126
Author: vozerov-gridgain <vo...@gridgain.com>
Date:   Thu Feb 4 11:18:51 2016 +0300

    Merge branch 'ignite-gc-opts' of https://github.com/yzhdanov/ignite into tmp_cumulative

    # Conflicts:
    #	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java

commit f095126588a5f6d6263ebe8cf2ec963ce37e7d77
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Wed Feb 3 17:49:14 2016 +0300

    https://issues.apache.org/jira/browse/IGNITE-2329 - single get

commit 7b987466af3fa0789e4cde8eb3f9a44126be129c
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Tue Feb 2 18:50:11 2016 +0300

    https://issues.apache.org/jira/browse/IGNITE-2329 - single get

commit 3e97d82d428ae5e672c4901cfc94224d19143c0f
Merge: 36c5500 4035d40
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Tue Feb 2 16:48:17 2016 +0300

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-gc-opts

commit 36c55008b355254e3944d7447bafa3078a51fe0d
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Fri Jan 29 14:38:54 2016 +0300

    Minor + merge from master

commit f8a55ad63d5f98e6c2663e35785dbf51beb412b2
Merge: dcdf2da d2a107b
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Fri Jan 29 14:36:08 2016 +0300

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-gc-opts

commit dcdf2da8323c2ebb9a4ad4a91823441e2bf58a14
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Fri Jan 29 10:04:03 2016 +0300

    Minor + merge from master

commit a5aa7e634a0d3140158b9c7fd40f62ddbcebf24a
Merge: 975966c 79db260
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Thu Jan 28 19:19:10 2016 +0300

    Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-gc-opts

commit 975966c96f05fb04ab759bc01ebef73f8727bc40
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Wed Jan 27 19:22:14 2016 +0300

    fixed rat

commit f85d2e68e9ea5c8af44d44c531be016beef102e1
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Wed Jan 27 18:42:41 2016 +0300

    fixed https://issues.apache.org/jira/browse/IGNITE-2329 + forceKeysFuture opts + disabled partition map

commit 914b365f79311e56c46ee4e214f60d60753030cc
Author: Yakov Zhdanov <yz...@gridgain.com>
Date:   Mon Jan 25 21:34:43 2016 +0300

    fixed https://issues.apache.org/jira/browse/IGNITE-2329 + forceKeysFuture opts


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

Branch: refs/heads/ignite-2329
Commit: e8a27685c507d728f4f2b553e1aeb01b8ed84d78
Parents: afd3bc1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 11:26:20 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 11:26:20 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java | 143 ++++++++++++++++---
 .../util/nio/GridSelectorNioSessionImpl.java    |   2 +-
 .../util/nio/SelectedSelectionKeySet.java       | 111 ++++++++++++++
 parent/pom.xml                                  |   1 +
 4 files changed, 237 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8a27685/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c7679c0..d42e771 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.nio;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
@@ -43,10 +44,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
@@ -102,6 +103,24 @@ public class GridNioServer<T> {
     /** SSL write buf limit. */
     private static final int WRITE_BUF_LIMIT = GridNioSessionMetaKey.nextUniqueKey();
 
+    /** */
+    private static final boolean DISABLE_KEYSET_OPTIMIZATION =
+        IgniteSystemProperties.getBoolean("IGNITE_NO_SELECTOR_OPTS", false);
+
+    /**
+     *
+     */
+    static {
+        // This is a workaround for JDK bug (NPE in Selector.open()).
+        // http://bugs.sun.com/view_bug.do?bug_id=6427854
+        try {
+            Selector.open().close();
+        }
+        catch (IOException ignored) {
+            // No-op.
+        }
+    }
+
     /** Accept worker thread. */
     @GridToStringExclude
     private final IgniteThread acceptThread;
@@ -184,17 +203,6 @@ public class GridNioServer<T> {
     /** Optional listener to monitor outbound message queue size. */
     private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
-    /** Static initializer ensures single-threaded execution of workaround. */
-    static {
-        // This is a workaround for JDK bug (NPE in Selector.open()).
-        // http://bugs.sun.com/view_bug.do?bug_id=6427854
-        try {
-            Selector.open().close();
-        }
-        catch (IOException ignored) {
-        }
-    }
-
     /**
      * @param addr Address.
      * @param port Port.
@@ -445,10 +453,8 @@ public class GridNioServer<T> {
             // Change from 0 to 1 means that worker thread should be waken up.
             clientWorkers.get(ses.selectorIndex()).offer(fut);
 
-        IgniteBiInClosure<GridNioSession, Integer> lsnr0 = msgQueueLsnr;
-
-        if (lsnr0 != null)
-            lsnr0.apply(ses, msgCnt);
+        if (msgQueueLsnr != null)
+            msgQueueLsnr.apply(ses, msgCnt);
     }
 
     /**
@@ -1239,6 +1245,9 @@ public class GridNioServer<T> {
         /** Selector to select read events. */
         private Selector selector;
 
+        /** Selected keys. */
+        private SelectedSelectionKeySet selectedKeys;
+
         /** Worker index. */
         private final int idx;
 
@@ -1253,7 +1262,7 @@ public class GridNioServer<T> {
             throws IgniteCheckedException {
             super(gridName, name, log);
 
-            selector = createSelector(null);
+            createSelector();
 
             this.idx = idx;
         }
@@ -1262,10 +1271,11 @@ public class GridNioServer<T> {
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             try {
                 boolean reset = false;
+
                 while (!closed) {
                     try {
                         if (reset)
-                            selector = createSelector(null);
+                            createSelector();
 
                         bodyInternal();
                     }
@@ -1290,6 +1300,50 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @throws IgniteCheckedException If failed.
+         */
+        private void createSelector() throws IgniteCheckedException {
+            selectedKeys = null;
+
+            selector = GridNioServer.this.createSelector(null);
+
+            if (DISABLE_KEYSET_OPTIMIZATION)
+                return;
+
+            try {
+                SelectedSelectionKeySet selectedKeySet = new SelectedSelectionKeySet();
+
+                Class<?> selectorImplClass =
+                    Class.forName("sun.nio.ch.SelectorImpl", false, U.gridClassLoader());
+
+                // Ensure the current selector implementation is what we can instrument.
+                if (!selectorImplClass.isAssignableFrom(selector.getClass()))
+                    return;
+
+                Field selectedKeysField = selectorImplClass.getDeclaredField("selectedKeys");
+                Field publicSelectedKeysField = selectorImplClass.getDeclaredField("publicSelectedKeys");
+
+                selectedKeysField.setAccessible(true);
+                publicSelectedKeysField.setAccessible(true);
+
+                selectedKeysField.set(selector, selectedKeySet);
+                publicSelectedKeysField.set(selector, selectedKeySet);
+
+                selectedKeys = selectedKeySet;
+
+                if (log.isDebugEnabled())
+                    log.debug("Instrumented an optimized java.util.Set into: " + selector);
+            }
+            catch (Exception e) {
+                selectedKeys = null;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to instrument an optimized java.util.Set into selector [selector=" + selector
+                        + ", err=" + e + ']');
+            }
+        }
+
+        /**
          * Adds socket channel to the registration queue and wakes up reading thread.
          *
          * @param req Change request.
@@ -1385,7 +1439,10 @@ public class GridNioServer<T> {
                     // Wake up every 2 seconds to check if closed.
                     if (selector.select(2000) > 0) {
                         // Walk through the ready keys collection and process network events.
-                        processSelectedKeys(selector.selectedKeys());
+                        if (selectedKeys == null)
+                            processSelectedKeys(selector.selectedKeys());
+                        else
+                            processSelectedKeysOptimized(selectedKeys.flip());
                     }
 
                     long now = U.currentTimeMillis();
@@ -1431,10 +1488,58 @@ public class GridNioServer<T> {
          * @param keys Selected keys.
          * @throws ClosedByInterruptException If this thread was interrupted while reading data.
          */
+        private void processSelectedKeysOptimized(SelectionKey[] keys) throws ClosedByInterruptException {
+            for (int i = 0; ; i ++) {
+                final SelectionKey key = keys[i];
+
+                if (key == null)
+                    break;
+
+                // null out entry in the array to allow to have it GC'ed once the Channel close
+                // See https://github.com/netty/netty/issues/2363
+                keys[i] = null;
+
+                // Was key closed?
+                if (!key.isValid())
+                    continue;
+
+                GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+                assert ses != null;
+
+                try {
+                    if (key.isReadable())
+                        processRead(key);
+
+                    if (key.isValid() && key.isWritable())
+                        processWrite(key);
+                }
+                catch (ClosedByInterruptException e) {
+                    // This exception will be handled in bodyInternal() method.
+                    throw e;
+                }
+                catch (Exception e) {
+                    if (!closed)
+                        U.warn(log, "Failed to process selector key (will close): " + ses, e);
+
+                    close(ses, new GridNioException(e));
+                }
+            }
+        }
+
+        /**
+         * Processes keys selected by a selector.
+         *
+         * @param keys Selected keys.
+         * @throws ClosedByInterruptException If this thread was interrupted while reading data.
+         */
         private void processSelectedKeys(Set<SelectionKey> keys) throws ClosedByInterruptException {
             if (log.isTraceEnabled())
                 log.trace("Processing keys in client worker: " + keys.size());
 
+            if (keys.isEmpty())
+                return;
+
             for (Iterator<SelectionKey> iter = keys.iterator(); iter.hasNext(); ) {
                 SelectionKey key = iter.next();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8a27685/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index deb7d2b..1241f99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -309,4 +309,4 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     @Override public String toString() {
         return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8a27685/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
new file mode 100644
index 0000000..d91f0e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2013 The Netty Project
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY 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.nio;
+
+
+import java.nio.channels.SelectionKey;
+import java.util.AbstractSet;
+import java.util.Iterator;
+
+final class SelectedSelectionKeySet extends AbstractSet<SelectionKey> {
+
+        private SelectionKey[] keysA;
+        private int keysASize;
+        private SelectionKey[] keysB;
+        private int keysBSize;
+        private boolean isA = true;
+
+    SelectedSelectionKeySet() {
+        keysA = new SelectionKey[1024];
+        keysB = keysA.clone();
+    }
+
+    @Override
+    public boolean add(SelectionKey o) {
+        if (o == null) {
+            return false;
+        }
+
+        if (isA) {
+            int size = keysASize;
+            keysA[size ++] = o;
+            keysASize = size;
+            if (size == keysA.length) {
+                doubleCapacityA();
+            }
+        } else {
+            int size = keysBSize;
+            keysB[size ++] = o;
+            keysBSize = size;
+            if (size == keysB.length) {
+                doubleCapacityB();
+            }
+        }
+
+        return true;
+    }
+
+    private void doubleCapacityA() {
+        SelectionKey[] newKeysA = new SelectionKey[keysA.length << 1];
+        System.arraycopy(keysA, 0, newKeysA, 0, keysASize);
+        keysA = newKeysA;
+    }
+
+    private void doubleCapacityB() {
+        SelectionKey[] newKeysB = new SelectionKey[keysB.length << 1];
+        System.arraycopy(keysB, 0, newKeysB, 0, keysBSize);
+        keysB = newKeysB;
+    }
+
+    SelectionKey[] flip() {
+        if (isA) {
+            isA = false;
+            keysA[keysASize] = null;
+            keysBSize = 0;
+            return keysA;
+        } else {
+            isA = true;
+            keysB[keysBSize] = null;
+            keysASize = 0;
+            return keysB;
+        }
+    }
+
+    @Override
+    public int size() {
+        if (isA) {
+            return keysASize;
+        } else {
+            return keysBSize;
+        }
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        return false;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return false;
+    }
+
+    @Override
+    public Iterator<SelectionKey> iterator() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e8a27685/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 53eefdd..437a30b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -780,6 +780,7 @@
                                         <exclude>**/NOTICE*</exclude>
                                         <exclude>**/LICENSE*</exclude>
                                         <exclude>src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java</exclude><!--BSD license-->
+                                        <exclude>src/main/java/org/apache/ignite/internal/util/nio/SelectedSelectionKeySet.java</exclude><!-- Apache 2.0 license -->
                                         <exclude>src/main/java/org/apache/ignite/internal/util/snaptree/*.java</exclude><!--BSD license-->
                                         <exclude>src/main/java/org/jsr166/*.java</exclude>
                                         <exclude>src/test/java/org/apache/ignite/p2p/p2p.properties</exclude><!--test depends on file content-->


[2/2] ignite git commit: IGNITE-2329: Minors.

Posted by vo...@apache.org.
IGNITE-2329: Minors.


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

Branch: refs/heads/ignite-2329
Commit: 54780f8c5ad16f6fb3f9ecb2b56f7ba98807c391
Parents: e8a2768
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 11:40:38 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 11:40:38 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  3 +
 .../ignite/internal/util/nio/GridNioServer.java | 58 ++++++++++----------
 2 files changed, 33 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/54780f8c/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index de7c10b..e0c68e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -367,6 +367,9 @@ public final class IgniteSystemProperties {
     /** JDBC driver cursor remove delay. */
     public static final String IGNITE_JDBC_DRIVER_CURSOR_REMOVE_DELAY = "IGNITE_JDBC_DRIVER_CURSOR_RMV_DELAY";
 
+    /** Flag indicating whether Ignite should not use aggressive optimization on Selector key set. */
+    public static final String IGNITE_NIO_SELECTOR_OPTIMIZATION_DISABLED = "IGNITE_NIO_SELECTOR_OPTIMIZATION_DISABLED";
+
     /**
      * Manages {@link OptimizedMarshaller} behavior of {@code serialVersionUID} computation for
      * {@link Serializable} classes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/54780f8c/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index d42e771..46d97b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -17,6 +17,31 @@
 
 package org.apache.ignite.internal.util.nio;
 
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+import sun.nio.ch.DirectBuffer;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.InetAddress;
@@ -44,31 +69,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.configuration.ConnectorConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
-import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter;
-import org.apache.ignite.internal.util.tostring.GridToStringExclude;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.LT;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.lang.IgniteBiInClosure;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.plugin.extensions.communication.Message;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.thread.IgniteThread;
-import org.jetbrains.annotations.Nullable;
-import sun.nio.ch.DirectBuffer;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_NIO_SELECTOR_OPTIMIZATION_DISABLED;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.ACK_CLOSURE;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.MSG_WRITER;
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPERATION;
@@ -103,9 +105,9 @@ public class GridNioServer<T> {
     /** SSL write buf limit. */
     private static final int WRITE_BUF_LIMIT = GridNioSessionMetaKey.nextUniqueKey();
 
-    /** */
-    private static final boolean DISABLE_KEYSET_OPTIMIZATION =
-        IgniteSystemProperties.getBoolean("IGNITE_NO_SELECTOR_OPTS", false);
+    /** Flag indicating whether Ignite should not use aggressive optimization on Selector keyset. */
+    private static final boolean DISABLE_SELECTOR_OPTIMIZATION =
+        IgniteSystemProperties.getBoolean(IGNITE_NIO_SELECTOR_OPTIMIZATION_DISABLED, false);
 
     /**
      *
@@ -1307,7 +1309,7 @@ public class GridNioServer<T> {
 
             selector = GridNioServer.this.createSelector(null);
 
-            if (DISABLE_KEYSET_OPTIMIZATION)
+            if (DISABLE_SELECTOR_OPTIMIZATION)
                 return;
 
             try {