You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/12/13 20:26:32 UTC

geode git commit: [GEODE-2144] Improve error message when no security credentials are provided.

Repository: geode
Updated Branches:
  refs/heads/develop 70800f526 -> 80525b7e8


[GEODE-2144] Improve error message when no security credentials are provided.

- Improve error message when no security credentials are provided.
- Add tests for the code paths exposing this message.
- Refactor making those code paths more testable.

This closes #310


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

Branch: refs/heads/develop
Commit: 80525b7e8414456fe18d1c4c55df48d9c3bf9f71
Parents: 70800f5
Author: Jared Stewart <js...@pivotal.io>
Authored: Fri Dec 9 13:29:36 2016 -0800
Committer: Kirk Lund <kl...@apache.org>
Committed: Tue Dec 13 10:06:22 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/geode/internal/Version.java |   3 +
 .../internal/cache/tier/sockets/AuthIds.java    |  48 +++++++
 .../internal/cache/tier/sockets/HandShake.java  |  38 +++---
 .../cache/tier/sockets/MessageIdExtractor.java  |  43 +++++++
 .../cache/tier/sockets/ServerConnection.java    |  83 ++++---------
 .../geode/internal/i18n/LocalizedStrings.java   |   4 +-
 .../apache/geode/internal/VersionJUnitTest.java |   8 ++
 .../cache/tier/sockets/HandShakeTest.java       |  52 ++++++++
 .../tier/sockets/MessageIdExtractorTest.java    |  83 +++++++++++++
 .../tier/sockets/ServerConnectionTest.java      | 124 +++++++++++++++++++
 10 files changed, 410 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/Version.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/Version.java b/geode-core/src/main/java/org/apache/geode/internal/Version.java
index 5f7c5ba..0f64e72 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/Version.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/Version.java
@@ -587,4 +587,7 @@ public final class Version implements Comparable<Version> {
     return bytes;
   }
 
+  public boolean isPre65() {
+    return compareTo(Version.GFE_65) < 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AuthIds.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AuthIds.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AuthIds.java
new file mode 100644
index 0000000..556cb36
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AuthIds.java
@@ -0,0 +1,48 @@
+/*
+ * 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.geode.internal.cache.tier.sockets;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+
+public class AuthIds {
+  private long connectionId;
+  private long uniqueId;
+
+  public AuthIds(byte[] bytes) throws Exception {
+    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
+    if (bytes.length == 8) {
+      // only connectionid
+      connectionId = dis.readLong();
+    } else if (bytes.length == 16) {
+      // first connectionId and then uniqueID
+      connectionId = dis.readLong();
+      uniqueId = dis.readLong();
+    } else {
+      throw new Exception("Auth ids are not in right form");
+    }
+  }
+
+
+  public long getConnectionId() {
+    return connectionId;
+  }
+
+  public long getUniqueId() {
+    return this.uniqueId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HandShake.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HandShake.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HandShake.java
index c0db945..9a5c6c6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HandShake.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/HandShake.java
@@ -245,6 +245,12 @@ public class HandShake implements ClientHandShake {
    */
   private static short overrideClientVersion = -1;
 
+  /** Constructor used for mocking */
+  protected HandShake() {
+    system = null;
+    id = null;
+  }
+
   /** Constructor used by server side connection */
   public HandShake(Socket sock, int timeout, DistributedSystem sys, Version clientVersion,
       byte communicationMode) throws IOException, AuthenticationRequiredException {
@@ -852,6 +858,18 @@ public class HandShake implements ClientHandShake {
     return _encrypt;
   }
 
+  /**
+   * Throws AuthenticationRequiredException if authentication is required but there are no
+   * credentials.
+   */
+  static void throwIfMissingRequiredCredentials(boolean requireAuthentication,
+      boolean hasCredentials) {
+    if (requireAuthentication && !hasCredentials) {
+      throw new AuthenticationRequiredException(
+          LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.toLocalizedString());
+    }
+  }
+
   // This assumes that authentication is the last piece of info in handshake
   public Properties readCredential(DataInputStream dis, DataOutputStream dos,
       DistributedSystem system) throws GemFireSecurityException, IOException {
@@ -860,14 +878,8 @@ public class HandShake implements ClientHandShake {
     boolean requireAuthentication = securityService.isClientSecurityRequired();
     try {
       byte secureMode = dis.readByte();
-      if (secureMode == CREDENTIALS_NONE) {
-        // when server is configured with authenticator, new joiner must
-        // pass its credentials.
-        if (requireAuthentication) {
-          throw new AuthenticationRequiredException(
-              LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED.toLocalizedString());
-        }
-      } else if (secureMode == CREDENTIALS_NORMAL) {
+      throwIfMissingRequiredCredentials(requireAuthentication, secureMode != CREDENTIALS_NONE);
+      if (secureMode == CREDENTIALS_NORMAL) {
         this.appSecureMode = CREDENTIALS_NORMAL;
         /*
          * if (requireAuthentication) { credentials = DataSerializer.readProperties(dis); } else {
@@ -1553,14 +1565,8 @@ public class HandShake implements ClientHandShake {
     Properties credentials = null;
     try {
       byte secureMode = dis.readByte();
-      if (secureMode == CREDENTIALS_NONE) {
-        // when server is configured with authenticator, new joiner must
-        // pass its credentials.
-        if (requireAuthentication) {
-          throw new AuthenticationRequiredException(
-              LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED.toLocalizedString());
-        }
-      } else if (secureMode == CREDENTIALS_NORMAL) {
+      throwIfMissingRequiredCredentials(requireAuthentication, secureMode != CREDENTIALS_NONE);
+      if (secureMode == CREDENTIALS_NORMAL) {
         if (requireAuthentication) {
           credentials = DataSerializer.readProperties(dis);
         } else {

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractor.java
new file mode 100644
index 0000000..311c074
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractor.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.geode.internal.cache.tier.sockets;
+
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.security.AuthenticationRequiredException;
+
+public class MessageIdExtractor {
+  public long getUniqueIdFromMessage(Message requestMessage, HandShake handshake, long connectionId)
+      throws AuthenticationRequiredException {
+    AuthIds aIds = getAuthIdsFromMessage(requestMessage, handshake);
+    if (connectionId != aIds.getConnectionId()) {
+      throw new AuthenticationRequiredException(
+          LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.toLocalizedString());
+    }
+    return aIds.getUniqueId();
+  }
+
+  private AuthIds getAuthIdsFromMessage(Message requestMessage, HandShake handshake)
+      throws AuthenticationRequiredException {
+    try {
+      byte[] secureBytes = requestMessage.getSecureBytes();
+      secureBytes = handshake.decryptBytes(secureBytes);
+      return new AuthIds(secureBytes);
+    } catch (Exception ex) {
+      throw new AuthenticationRequiredException(
+          LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.toLocalizedString(), ex);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
index 59b0fa4..ecd9c7a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnection.java
@@ -238,6 +238,8 @@ public class ServerConnection implements Runnable {
 
   private Principal principal;
 
+  private MessageIdExtractor messageIdExtractor = new MessageIdExtractor();
+
   /**
    * A debug flag used for testing Backward compatibility
    */
@@ -395,6 +397,10 @@ public class ServerConnection implements Runnable {
     this.keySetResponseMsg.setVersion(v);
   }
 
+  public void setRequestMsg(Message requestMsg) {
+    this.requestMsg = requestMsg;
+  }
+
   public Version getClientVersion() {
     return this.handshake.getVersion();
   }
@@ -1441,8 +1447,7 @@ public class ServerConnection implements Runnable {
     if (isClosed()) {
       return false;
     }
-    if (this.communicationMode == Acceptor.CLIENT_TO_SERVER
-        || this.communicationMode == Acceptor.GATEWAY_TO_GATEWAY
+    if (this.communicationMode == Acceptor.CLIENT_TO_SERVER || isGatewayConnection()
         || this.communicationMode == Acceptor.MONITOR_TO_SERVER
     /* || this.communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE */) {
       getAcceptor().decClientServerCnxCount();
@@ -1703,34 +1708,6 @@ public class ServerConnection implements Runnable {
     this.userAuthId = uniqueId;
   }
 
-  private static class AuthIds {
-    private long connectionId;
-    private long uniqueId;
-
-    public AuthIds(byte[] bytes) throws Exception {
-      DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes));
-      if (bytes.length == 8) {
-        // only connectionid
-        connectionId = dis.readLong();
-      } else if (bytes.length == 16) {
-        // first connectionId and then uniqueID
-        connectionId = dis.readLong();
-        uniqueId = dis.readLong();
-      } else {
-        throw new Exception("Auth ids are not in right form");
-      }
-    }
-
-
-    public long getConnectionId() {
-      return connectionId;
-    }
-
-    public long getUniqueId() {
-      return this.uniqueId;
-    }
-  }
-
   private byte[] encryptId(long id, ServerConnection servConn) throws Exception {
     // deserialize this using handshake keys
     HeapDataOutputStream hdos = null;
@@ -1748,40 +1725,22 @@ public class ServerConnection implements Runnable {
   public long getUniqueId() {
     long uniqueId = 0;
 
-    if (this.handshake.getVersion().compareTo(Version.GFE_65) < 0
-        || this.communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
+    if (this.handshake.getVersion().isPre65() || isGatewayConnection()) {
       uniqueId = this.userAuthId;
+    } else if (this.requestMsg.isSecureMode()) {
+      uniqueId = messageIdExtractor.getUniqueIdFromMessage(this.requestMsg,
+          (HandShake) this.handshake, this.connectionId);
     } else {
-      try {
-        // this.logger.fine("getAuthzRequest() isSecureMode = " + this.requestMsg.isSecureMode());
-        if (this.requestMsg.isSecureMode()) {
-          // get uniqueID from message
-          byte[] secureBytes = this.requestMsg.getSecureBytes();
-
-          secureBytes = ((HandShake) this.handshake).decryptBytes(secureBytes);
-          AuthIds aIds = new AuthIds(secureBytes);
-
-          if (this.connectionId != aIds.getConnectionId()) {
-            throw new AuthenticationRequiredException(
-                LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED.toLocalizedString());
-          } else {
-            uniqueId = aIds.getUniqueId();
-          }
-
-        } else {
-          throw new AuthenticationRequiredException(
-              LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED.toLocalizedString());
-        }
-      } catch (AuthenticationRequiredException are) {
-        throw are;
-      } catch (Exception ex) {
-        throw new AuthenticationRequiredException(
-            LocalizedStrings.HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED.toLocalizedString());
-      }
+      throw new AuthenticationRequiredException(
+          LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.toLocalizedString());
     }
     return uniqueId;
   }
 
+  private boolean isGatewayConnection() {
+    return getCommunicationMode() == Acceptor.GATEWAY_TO_GATEWAY;
+  }
+
   public AuthorizeRequest getAuthzRequest() throws AuthenticationRequiredException, IOException {
     // look client version and return authzrequest
     // for backward client it will be store in member variable userAuthId
@@ -1862,4 +1821,12 @@ public class ServerConnection implements Runnable {
   public void setClientDisconnectedException(Throwable e) {
     this.clientDisconnectedException = e;
   }
+
+  public void setMessageIdExtractor(MessageIdExtractor messageIdExtractor) {
+    this.messageIdExtractor = messageIdExtractor;
+  }
+
+  public MessageIdExtractor getMessageIdExtractor() {
+    return this.messageIdExtractor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index 38f7397..48b8a14 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -5350,8 +5350,8 @@ public class LocalizedStrings {
       new StringId(4192, "Server expecting SSL connection");
   public static final StringId HandShake_FAILED_TO_ACQUIRE_AUTHINITIALIZE_METHOD_0 =
       new StringId(4194, "Failed to acquire AuthInitialize method {0}");
-  public static final StringId HandShake_NO_SECURITY_PROPERTIES_ARE_PROVIDED =
-      new StringId(4195, "No security-* properties are provided");
+  public static final StringId HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED =
+      new StringId(4195, "No security credentials are provided");
   public static final StringId HandShake_FAILURE_IN_READING_CREDENTIALS =
       new StringId(4196, "Failure in reading credentials");
   public static final StringId HandShake_FAILED_TO_ACQUIRE_AUTHENTICATOR_OBJECT =

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/test/java/org/apache/geode/internal/VersionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/VersionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/VersionJUnitTest.java
index 0bc6822..8cd4d81 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/VersionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/VersionJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal;
 
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;
@@ -45,4 +46,11 @@ public class VersionJUnitTest {
     assertTrue(later.compareTo(later.ordinal()) == 0);
     assertTrue(earlier.compareTo(later.ordinal()) < 0);
   }
+
+  @Test
+  public void testIsPre65() {
+    assertTrue(Version.GFE_61.isPre65());
+    assertFalse(Version.GFE_65.isPre65());
+    assertFalse(Version.GFE_70.isPre65());
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/HandShakeTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/HandShakeTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/HandShakeTest.java
new file mode 100644
index 0000000..2850500
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/HandShakeTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.geode.internal.cache.tier.sockets;
+
+import static org.apache.geode.internal.i18n.LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class HandShakeTest {
+
+  @Test
+  public void authRequiredHasCredentials() throws Exception {
+    HandShake.throwIfMissingRequiredCredentials(true, true);
+  }
+
+  @Test
+  public void authRequiredHasNoCredentials() throws Exception {
+    assertThatThrownBy(() -> HandShake.throwIfMissingRequiredCredentials(true, false))
+        .isExactlyInstanceOf(AuthenticationRequiredException.class)
+        .hasMessage(HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.toLocalizedString());
+  }
+
+  @Test
+  public void authNotRequiredHasCredentials() throws Exception {
+    HandShake.throwIfMissingRequiredCredentials(false, true);
+  }
+
+  @Test
+  public void authNotRequiredHasNoCredentials() throws Exception {
+    HandShake.throwIfMissingRequiredCredentials(false, false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractorTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractorTest.java
new file mode 100644
index 0000000..0bc6381
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/MessageIdExtractorTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.geode.internal.cache.tier.sockets;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.when;
+
+import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+@Category(UnitTest.class)
+public class MessageIdExtractorTest {
+  @Mock
+  Message requestMessage;
+
+  @Mock
+  HandShake handshake;
+
+  private MessageIdExtractor messageIdExtractor;
+
+  private Long connectionId = 123L;
+  private Long uniqueId = 234L;
+  private byte[] decryptedBytes;
+
+  @Before
+  public void before() throws Exception {
+    this.messageIdExtractor = new MessageIdExtractor();
+    decryptedBytes = byteArrayFromIds(connectionId, uniqueId);
+
+    MockitoAnnotations.initMocks(this);
+
+    when(handshake.decryptBytes(any())).thenReturn(decryptedBytes);
+  }
+
+  @Test
+  public void getUniqueIdFromMessage() throws Exception {
+    assertThat(messageIdExtractor.getUniqueIdFromMessage(requestMessage, handshake, connectionId))
+        .isEqualTo(uniqueId);
+  }
+
+  @Test
+  public void throwsWhenConnectionIdsDoNotMatch() throws Exception {
+    long otherConnectionId = 789L;
+
+    assertThatThrownBy(() -> messageIdExtractor.getUniqueIdFromMessage(requestMessage, handshake,
+        otherConnectionId)).isInstanceOf(AuthenticationRequiredException.class);
+  }
+
+  private byte[] byteArrayFromIds(Long connectionId, Long uniqueId) throws IOException {
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutputStream dis = new DataOutputStream(byteArrayOutputStream);
+    dis.writeLong(connectionId);
+    dis.writeLong(uniqueId);
+    dis.flush();
+    return byteArrayOutputStream.toByteArray();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/80525b7e/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
new file mode 100644
index 0000000..153e77b
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.geode.internal.cache.tier.sockets;
+
+
+import static org.apache.geode.internal.i18n.LocalizedStrings.HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.internal.Version;
+import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.net.InetAddress;
+import java.net.Socket;
+
+@Category(UnitTest.class)
+public class ServerConnectionTest {
+  @Mock
+  private Message requestMsg;
+
+  @Mock
+  private HandShake handshake;
+
+  @Mock
+  private MessageIdExtractor messageIdExtractor;
+
+  @InjectMocks
+  private ServerConnection serverConnection;
+
+  @Before
+  public void setUp() {
+    AcceptorImpl acceptor = mock(AcceptorImpl.class);
+
+    InetAddress inetAddress = mock(InetAddress.class);
+    when(inetAddress.getHostAddress()).thenReturn("localhost");
+
+    Socket socket = mock(Socket.class);
+    when(socket.getInetAddress()).thenReturn(inetAddress);
+
+    Cache cache = mock(Cache.class);
+
+    serverConnection = new ServerConnection(socket, cache, null, null, 0, 0, null,
+        Acceptor.PRIMARY_SERVER_TO_CLIENT, acceptor);
+    MockitoAnnotations.initMocks(this);
+  }
+
+
+  @Test
+  public void pre65SecureShouldReturnUserAuthId() {
+    long userAuthId = 12345L;
+    serverConnection.setUserAuthId(userAuthId);
+
+    when(handshake.getVersion()).thenReturn(Version.GFE_61);
+    when(requestMsg.isSecureMode()).thenReturn(true);
+
+    assertThat(serverConnection.getUniqueId()).isEqualTo(userAuthId);
+  }
+
+  @Test
+  public void pre65NonSecureShouldReturnUserAuthId() {
+    long userAuthId = 12345L;
+    serverConnection.setUserAuthId(userAuthId);
+
+    when(handshake.getVersion()).thenReturn(Version.GFE_61);
+    when(requestMsg.isSecureMode()).thenReturn(false);
+
+    assertThat(serverConnection.getUniqueId()).isEqualTo(userAuthId);
+  }
+
+
+  @Test
+  public void post65SecureShouldUseUniqueIdFromMessage() {
+    long uniqueIdFromMessage = 23456L;
+    when(handshake.getVersion()).thenReturn(Version.GFE_82);
+    serverConnection.setRequestMsg(requestMsg);
+
+    assertThat(serverConnection.getRequestMessage()).isSameAs(requestMsg);
+    when(requestMsg.isSecureMode()).thenReturn(true);
+
+    when(messageIdExtractor.getUniqueIdFromMessage(any(Message.class), any(HandShake.class),
+        anyLong())).thenReturn(uniqueIdFromMessage);
+    serverConnection.setMessageIdExtractor(messageIdExtractor);
+
+    assertThat(serverConnection.getUniqueId()).isEqualTo(uniqueIdFromMessage);
+  }
+
+  @Test
+  public void post65NonSecureShouldThrow() {
+    when(handshake.getVersion()).thenReturn(Version.GFE_82);
+    when(requestMsg.isSecureMode()).thenReturn(false);
+
+    assertThatThrownBy(serverConnection::getUniqueId)
+        .isExactlyInstanceOf(AuthenticationRequiredException.class)
+        .hasMessage(HandShake_NO_SECURITY_CREDENTIALS_ARE_PROVIDED.getRawText());
+  }
+
+}