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/11/09 08:38:52 UTC

[38/50] [abbrv] ignite git commit: IGNITE-4110 - Fixed BUFFER_UNDERFLOW and BUFFER_OVERFLOW handling in BlockingSslHandler

IGNITE-4110 - Fixed BUFFER_UNDERFLOW and BUFFER_OVERFLOW handling in BlockingSslHandler


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

Branch: refs/heads/master
Commit: 9ebbaea586d9ba360a1325a840fc7d81c93a95fc
Parents: 087f640
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Oct 28 16:08:44 2016 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Oct 28 16:08:44 2016 -0700

----------------------------------------------------------------------
 .../util/nio/ssl/BlockingSslHandler.java        | 50 ++++++++++----------
 ...CommunicationSpiSslSmallBuffersSelfTest.java | 43 +++++++++++++++++
 .../IgniteSpiCommunicationSelfTestSuite.java    |  7 ++-
 3 files changed, 75 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9ebbaea5/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
index b3e8201..638106f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
@@ -35,6 +35,7 @@ import static javax.net.ssl.SSLEngineResult.HandshakeStatus.FINISHED;
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_TASK;
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_UNWRAP;
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NOT_HANDSHAKING;
+import static javax.net.ssl.SSLEngineResult.Status.BUFFER_OVERFLOW;
 import static javax.net.ssl.SSLEngineResult.Status.BUFFER_UNDERFLOW;
 import static javax.net.ssl.SSLEngineResult.Status.CLOSED;
 import static javax.net.ssl.SSLEngineResult.Status.OK;
@@ -92,7 +93,10 @@ public class BlockingSslHandler {
         this.order = order;
 
         // Allocate a little bit more so SSL engine would not return buffer overflow status.
-        int netBufSize = sslEngine.getSession().getPacketBufferSize() + 50;
+        //
+        // System property override is for test purposes only.
+        int netBufSize = Integer.getInteger("BlockingSslHandler.netBufSize",
+            sslEngine.getSession().getPacketBufferSize() + 50);
 
         outNetBuf = directBuf ? ByteBuffer.allocateDirect(netBufSize) : ByteBuffer.allocate(netBufSize);
         outNetBuf.order(order);
@@ -173,15 +177,22 @@ public class BlockingSslHandler {
 
                     SSLEngineResult res = sslEngine.wrap(handshakeBuf, outNetBuf);
 
-                    outNetBuf.flip();
+                    if (res.getStatus() == BUFFER_OVERFLOW) {
+                        outNetBuf = expandBuffer(outNetBuf, outNetBuf.capacity() * 2);
+
+                        outNetBuf.flip();
+                    }
+                    else {
+                        outNetBuf.flip();
+
+                        writeNetBuffer();
+                    }
 
                     handshakeStatus = res.getHandshakeStatus();
 
                     if (log.isDebugEnabled())
                         log.debug("Wrapped handshake data [status=" + res.getStatus() + ", handshakeStatus=" +
-                        handshakeStatus + ']');
-
-                    writeNetBuffer();
+                            handshakeStatus + ']');
 
                     break;
                 }
@@ -368,6 +379,7 @@ public class BlockingSslHandler {
         inNetBuf.flip();
 
         SSLEngineResult res = unwrap0();
+
         handshakeStatus = res.getHandshakeStatus();
 
         checkStatus(res);
@@ -384,6 +396,11 @@ public class BlockingSslHandler {
 
             renegotiateIfNeeded(res);
         }
+        else if (res.getStatus() == BUFFER_UNDERFLOW) {
+            inNetBuf.compact();
+
+            inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2);
+        }
         else
             // prepare to be written again
             inNetBuf.compact();
@@ -462,6 +479,7 @@ public class BlockingSslHandler {
         int appBufSize = Math.max(sslEngine.getSession().getApplicationBufferSize() + 50, netBufSize * 2);
 
         ByteBuffer buf = ByteBuffer.allocate(appBufSize);
+
         buf.order(order);
 
         return buf;
@@ -504,9 +522,9 @@ public class BlockingSslHandler {
      * @return Expanded byte buffer.
      */
     private ByteBuffer expandBuffer(ByteBuffer original, int cap) {
-        ByteBuffer res = ByteBuffer.allocate(cap);
+        ByteBuffer res = original.isDirect() ? ByteBuffer.allocateDirect(cap) : ByteBuffer.allocate(cap);
 
-        res.order(ByteOrder.nativeOrder());
+        res.order(original.order());
 
         original.flip();
 
@@ -514,20 +532,4 @@ public class BlockingSslHandler {
 
         return res;
     }
-
-    /**
-     * Copies the given byte buffer.
-     *
-     * @param original Byte buffer to copy.
-     * @return Copy of the original byte buffer.
-     */
-    private ByteBuffer copy(ByteBuffer original) {
-        ByteBuffer cp = ByteBuffer.allocate(original.remaining());
-
-        cp.put(original);
-
-        cp.flip();
-
-        return cp;
-    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9ebbaea5/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java
new file mode 100644
index 0000000..1ccb9bb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler;
+import org.apache.ignite.testframework.junits.spi.GridSpiTest;
+
+/**
+ * Copy of {@link GridTcpCommunicationSpiSslSelfTest}, but overriding initial buffer
+ * sizes in {@link BlockingSslHandler}. This checks that {@code BUFFER_UNDERFLOW} and
+ * {@code BUFFER_OVERFLOW} conditions are properly handled.
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiSslSmallBuffersSelfTest extends GridTcpCommunicationSpiSslSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        System.setProperty("BlockingSslHandler.netBufSize", "1000");
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        System.clearProperty("BlockingSslHandler.netBufSize");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9ebbaea5/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
index eb88524..c557fbb 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
@@ -28,6 +28,8 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryFa
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest;
+import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSelfTest;
+import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSmallBuffersSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiStartStopSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpFailureDetectionSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpNoDelayOffSelfTest;
@@ -53,6 +55,9 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSslSelfTest.class));
 
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSmallBuffersSelfTest.class));
+
         suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiShmemSelfTest.class));
@@ -69,4 +74,4 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}