You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2017/09/01 19:12:55 UTC

geode git commit: GEODE-3059: LoadMonitor.connectionClosed incrementing statistics only for client-server connection

Repository: geode
Updated Branches:
  refs/heads/develop 836451a5a -> fecec42fb


GEODE-3059: LoadMonitor.connectionClosed incrementing statistics only for client-server connection

The stat should only be incremented/decremented for certain kinds of
connections.  I've modified it to include protobuf connections.  All
of the constant byte identifiers in Acceptor.java have been moved to
an enum in CommunicationMode.java.  In that class I've added some
"isa" checks to replace the many big "if" checks for different kinds
of connection modes.

A new connection modes will henceforth need to be added to
CommunicationMode.java where the appropriate "isa" methods can be
updated to include the new mode.


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

Branch: refs/heads/develop
Commit: fecec42fb57630a669b653ccb54a97b42948d219
Parents: 836451a
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Sep 1 12:10:39 2017 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Sep 1 12:12:49 2017 -0700

----------------------------------------------------------------------
 .../client/internal/ConnectionFactoryImpl.java  |  10 +-
 .../cache/client/internal/ConnectionImpl.java   |   4 +-
 .../cache/server/internal/LoadMonitor.java      |  13 +-
 .../internal/tcpserver/TcpServer.java           |   4 +-
 .../geode/internal/cache/tier/Acceptor.java     |  49 ------
 .../internal/cache/tier/ClientHandShake.java    |   2 +-
 .../internal/cache/tier/CommunicationMode.java  | 155 +++++++++++++++++++
 .../cache/tier/sockets/AcceptorImpl.java        |  41 ++---
 .../cache/tier/sockets/CacheClientNotifier.java |  13 +-
 .../cache/tier/sockets/ClientHealthMonitor.java |   3 +-
 .../cache/tier/sockets/ConnectionListener.java  |   6 +-
 .../tier/sockets/ConnectionListenerAdapter.java |   6 +-
 .../internal/cache/tier/sockets/HandShake.java  |  57 +++----
 .../cache/tier/sockets/ServerConnection.java    |  39 ++---
 .../tier/sockets/ServerConnectionFactory.java   |   6 +-
 .../tier/sockets/ServerHandShakeProcessor.java  |   5 +-
 .../cache/server/internal/LoadMonitorTest.java  |  50 ++++++
 .../GenericProtocolServerConnectionTest.java    |   6 +-
 .../sockets/ServerConnectionFactoryTest.java    |  38 ++---
 .../tier/sockets/ServerConnectionTest.java      |   8 +-
 .../geode/codeAnalysis/excludedClasses.txt      |   1 +
 .../protocol/AuthenticationIntegrationTest.java |  12 +-
 .../RoundTripLocatorConnectionJUnitTest.java    |   4 +-
 23 files changed, 343 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionFactoryImpl.java
index dea8644..62bf8fd 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionFactoryImpl.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.client.internal.ServerBlackList.FailureTracker;
 import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientUpdater;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.HandShake;
@@ -110,13 +110,13 @@ public class ConnectionFactoryImpl implements ConnectionFactory {
     blackList.start(background);
   }
 
-  private byte getCommMode(boolean forQueue) {
+  private CommunicationMode getCommMode(boolean forQueue) {
     if (this.usedByGateway || (this.gatewaySender != null)) {
-      return Acceptor.GATEWAY_TO_GATEWAY;
+      return CommunicationMode.GatewayToGateway;
     } else if (forQueue) {
-      return Acceptor.CLIENT_TO_SERVER_FOR_QUEUE;
+      return CommunicationMode.ClientToServerForQueue;
     } else {
-      return Acceptor.CLIENT_TO_SERVER;
+      return CommunicationMode.ClientToServer;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
index 078844f..81e878e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
@@ -34,6 +34,7 @@ import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.sockets.HandShake;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.cache.tier.sockets.ServerQueueStatus;
@@ -98,7 +99,8 @@ public class ConnectionImpl implements Connection {
 
   public ServerQueueStatus connect(EndpointManager endpointManager, ServerLocation location,
       HandShake handShake, int socketBufferSize, int handShakeTimeout, int readTimeout,
-      byte communicationMode, GatewaySender sender, SocketCreator sc) throws IOException {
+      CommunicationMode communicationMode, GatewaySender sender, SocketCreator sc)
+      throws IOException {
     theSocket = sc.connectForClient(location.getHostName(), location.getPort(), handShakeTimeout,
         socketBufferSize);
     theSocket.setTcpNoDelay(true);

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/cache/server/internal/LoadMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/server/internal/LoadMonitor.java b/geode-core/src/main/java/org/apache/geode/cache/server/internal/LoadMonitor.java
index 1c571a9..92be55a 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/server/internal/LoadMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/server/internal/LoadMonitor.java
@@ -27,7 +27,7 @@ import org.apache.geode.cache.server.ServerLoadProbe;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
 import org.apache.geode.internal.cache.CacheServerAdvisor;
-import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
@@ -45,7 +45,7 @@ public class LoadMonitor implements ConnectionListener {
   private static final Logger logger = LogService.getLogger();
 
   private final ServerLoadProbe probe;
-  private final ServerMetricsImpl metrics;
+  protected final ServerMetricsImpl metrics;
   protected final CacheServerAdvisor advisor;
   protected ServerLocation location;
   private final PollingThread pollingThread;
@@ -88,8 +88,8 @@ public class LoadMonitor implements ConnectionListener {
     probe.close();
   }
 
-  public void connectionClosed(boolean lastConnection, byte communicationMode) {
-    if (communicationMode == Acceptor.CLIENT_TO_SERVER) {
+  public void connectionClosed(boolean lastConnection, CommunicationMode communicationMode) {
+    if (communicationMode.isClientOperations()) {
       metrics.decConnectionCount();
     }
     if (lastConnection) {
@@ -101,9 +101,8 @@ public class LoadMonitor implements ConnectionListener {
     return lastLoad;
   }
 
-  public void connectionOpened(boolean firstConnection, byte communicationMode) {
-    // ignore all other types of client connections.
-    if (communicationMode == Acceptor.CLIENT_TO_SERVER) {
+  public void connectionOpened(boolean firstConnection, CommunicationMode communicationMode) {
+    if (communicationMode.isClientOperations()) {
       metrics.incConnectionCount();
     }
     if (firstConnection) {

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
index 83f87ee..896610e 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
@@ -57,7 +57,7 @@ import org.apache.geode.internal.Version;
 import org.apache.geode.internal.VersionedDataInputStream;
 import org.apache.geode.internal.VersionedDataOutputStream;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.sockets.ClientProtocolMessageHandler;
 import org.apache.geode.internal.cache.tier.sockets.HandShake;
 import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
@@ -501,7 +501,7 @@ public class TcpServer {
     // read the first byte & check for an improperly configured client pool trying
     // to contact a cache server
     int firstByte = input.readUnsignedByte();
-    if (firstByte >= Acceptor.CLIENT_TO_SERVER) {
+    if (CommunicationMode.isValidMode(firstByte)) {
       sock.getOutputStream().write(HandShake.REPLY_SERVER_IS_LOCATOR);
       throw new Exception("Improperly configured client detected - use addPoolLocator to "
           + "configure its locators instead of addPoolServer.");

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/Acceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/Acceptor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/Acceptor.java
index e12a409..f062129 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/Acceptor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/Acceptor.java
@@ -27,55 +27,6 @@ import org.apache.geode.internal.Version;
  */
 public abstract class Acceptor {
 
-  // The following are communications "mode" bytes sent as the first byte of a
-  // client/server handshake. They must not be larger than 1 byte
-
-  /**
-   * Byte meaning that the Socket is being used for 'client to server' communication.
-   */
-  public static final byte CLIENT_TO_SERVER = (byte) 100;
-
-  /**
-   * Byte meaning that the Socket is being used for 'primary server to client' communication.
-   */
-  public static final byte PRIMARY_SERVER_TO_CLIENT = (byte) 101;
-
-  /**
-   * Byte meaning that the Socket is being used for 'seconadry server to client' communication.
-   */
-  public static final byte SECONDARY_SERVER_TO_CLIENT = (byte) 102;
-
-  /**
-   * Byte meaning that the Socket is being used for 'gateway to gateway' communication.
-   */
-  public static final byte GATEWAY_TO_GATEWAY = (byte) 103;
-
-  /**
-   * Byte meaning that the Socket is being used for 'monitor to gateway' communication.
-   */
-  public static final byte MONITOR_TO_SERVER = (byte) 104;
-
-  /**
-   * Byte meaning that the connection between the server and client was successful.
-   */
-  public static final byte SUCCESSFUL_SERVER_TO_CLIENT = (byte) 105;
-
-  /**
-   * Byte meaning that the connection between the server and client was unsuccessful.
-   */
-  public static final byte UNSUCCESSFUL_SERVER_TO_CLIENT = (byte) 106;
-
-  /**
-   * Byte meaning that the Socket is being used for 'client to server' messages related to a client
-   * queue (register interest, create cq, etc.).
-   */
-  public static final byte CLIENT_TO_SERVER_FOR_QUEUE = (byte) 107;
-
-  /**
-   * For the new client-server protocol, which ignores the usual handshake mechanism.
-   */
-  public static final byte PROTOBUF_CLIENT_SERVER_PROTOCOL = (byte) 110;
-
   /**
    * The GFE version of the server.
    * 

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/ClientHandShake.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/ClientHandShake.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/ClientHandShake.java
index f7a39f3..3e0332b 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/ClientHandShake.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/ClientHandShake.java
@@ -40,5 +40,5 @@ public interface ClientHandShake {
   public Version getVersion();
 
   public void accept(OutputStream out, InputStream in, byte epType, int qSize,
-      byte communicationMode, Principal principal) throws IOException;
+      CommunicationMode communicationMode, Principal principal) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/CommunicationMode.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/CommunicationMode.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/CommunicationMode.java
new file mode 100644
index 0000000..9abf2f3
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/CommunicationMode.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+/**
+ * The following are communications "mode" bytes sent as the first byte of a client/server
+ * handshake. They must not be larger than 1 byte.
+ * <p>
+ * NOTE: new protobuf modes should be added as new values in this enumeration and the appropriate
+ * query methods must be updated.
+ */
+public enum CommunicationMode {
+  /**
+   * Byte meaning that the Socket is being used for 'client to server' communication.
+   */
+  ClientToServer((byte) 100, "client"),
+  /**
+   * Byte meaning that the Socket is being used for 'primary server to client' communication.
+   */
+  PrimaryServerToClient((byte) 101, "primary server to client"),
+  /**
+   * Byte meaning that the Socket is being used for 'secondary server to client' communication.
+   */
+  SecondaryServerToClient((byte) 102, "secondary server to client"),
+  /**
+   * Byte meaning that the Socket is being used for 'gateway to gateway' communication.
+   */
+  GatewayToGateway((byte) 103, "gateway"),
+  /**
+   * Byte meaning that the Socket is being used for 'monitor to gateway' communication.
+   */
+  MonitorToServer((byte) 104, "monitor"),
+  /**
+   * Byte meaning that the connection between the server and client was successful. This is not
+   * actually a communication mode but an acknowledgement byte.
+   */
+  SuccessfulServerToClient((byte) 105, "successful server to client"),
+  /**
+   * Byte meaning that the connection between the server and client was unsuccessful. This is not
+   * actually a communication mode but an error byte.
+   */
+  UnsuccessfulServerToClient((byte) 106, "unsucessful server to client"),
+  /**
+   * Byte meaning that the Socket is being used for 'client to server' messages related to a client
+   * queue (register interest, create cq, etc.).
+   */
+  ClientToServerForQueue((byte) 107, "clientToServerForQueue"),
+  /**
+   * For the new client-server protocol, which ignores the usual handshake mechanism.
+   */
+  ProtobufClientServerProtocol((byte) 110, "Protobuf client");
+
+  /**
+   * is this a client-initiated operations connection?
+   */
+  public boolean isClientOperations() {
+    return this == ClientToServer || this == ProtobufClientServerProtocol;
+  }
+
+  /**
+   * is this any type of client/server connection?
+   */
+  public boolean isClientToServerOrSubscriptionFeed() {
+    return this == ClientToServer || this == PrimaryServerToClient
+        || this == SecondaryServerToClient || this == ClientToServerForQueue
+        || this == ProtobufClientServerProtocol;
+  }
+
+  /**
+   * be the first to describe this method
+   */
+  public boolean isSubscriptionFeed() {
+    return this == PrimaryServerToClient || this == SecondaryServerToClient;
+  }
+
+  /**
+   * is this connection counted in the ClientServerCnxCount statistic?
+   */
+  public boolean isCountedAsClientServerConnection() {
+    return this == ClientToServer || this == MonitorToServer
+        || this == ProtobufClientServerProtocol;
+  }
+
+  /**
+   * is this a WAN connection?
+   */
+  public boolean isWAN() {
+    return this == GatewayToGateway;
+  }
+
+  /**
+   * The modeNumber is the byte written on-wire that indicates this connection mode
+   */
+  private byte modeNumber;
+
+  private String description;
+
+  CommunicationMode(byte mode, String description) {
+    modeNumber = mode;
+    this.description = description;
+  }
+
+  public byte getModeNumber() {
+    return this.modeNumber;
+  }
+
+  /**
+   * check the given mode to see if it is assigned to one of the enumeration's instances
+   */
+  public static boolean isValidMode(int mode) {
+    return 100 <= mode && mode <= 110;
+  }
+
+  public static CommunicationMode fromModeNumber(byte modeNumber) {
+    switch (modeNumber) {
+      case 100:
+        return ClientToServer;
+      case 101:
+        return PrimaryServerToClient;
+      case 102:
+        return SecondaryServerToClient;
+      case 103:
+        return GatewayToGateway;
+      case 104:
+        return MonitorToServer;
+      case 105:
+        return SuccessfulServerToClient;
+      case 106:
+        return UnsuccessfulServerToClient;
+      case 107:
+        return ClientToServerForQueue;
+      case 110:
+        return ProtobufClientServerProtocol;
+      default:
+        throw new IllegalArgumentException("unknown communications mode: " + modeNumber);
+    }
+  }
+
+  public String toString() {
+    return this.description;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
index 2e33af8..fa36fab 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
@@ -16,6 +16,7 @@
 package org.apache.geode.internal.cache.tier.sockets;
 
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_ACCESSOR_PP;
+import static org.apache.geode.internal.cache.tier.CommunicationMode.ClientToServerForQueue;
 
 import org.apache.geode.CancelException;
 import org.apache.geode.SystemFailure;
@@ -39,6 +40,7 @@ import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.partitioned.AllBucketProfilesUpdateMessage;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.wan.GatewayReceiverStats;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -89,7 +91,6 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import javax.net.ssl.SSLException;
 
 /**
  * Implements the acceptor thread on the bridge server. Accepts connections from the edge and starts
@@ -1440,21 +1441,30 @@ public class AcceptorImpl extends Acceptor implements Runnable, CommBufferPool {
 
     socket.setTcpNoDelay(this.tcpNoDelay);
 
+    final CommunicationMode mode;
+    try {
+      mode = CommunicationMode.fromModeNumber(communicationMode);
+    } catch (IllegalArgumentException e) {
+      // possible if a client uses SSL & the server isn't configured to use SSL
+      logger.warn("Error processing client connection", e);
+      throw new EOFException();
+    }
+
     String communicationModeStr;
-    if (communicationMode == PRIMARY_SERVER_TO_CLIENT
-        || communicationMode == SECONDARY_SERVER_TO_CLIENT) {
-      boolean primary = communicationMode == PRIMARY_SERVER_TO_CLIENT;
+    if (mode.isSubscriptionFeed()) {
+      boolean primary = mode == CommunicationMode.PrimaryServerToClient;
       logger.debug(":Bridge server: Initializing {} server-to-client communication socket: {}",
           primary ? "primary" : "secondary", socket);
       AcceptorImpl.this.clientNotifier.registerClient(socket, primary, this.acceptorId,
           this.notifyBySubscription);
       return;
     }
-    communicationModeStr = getCommunicationMode(communicationMode);
+    communicationModeStr = mode.toString();
 
     logger.debug("Bridge server: Initializing {} communication socket: {}", communicationModeStr,
         socket);
-    if (communicationMode != CLIENT_TO_SERVER_FOR_QUEUE) {
+    boolean notForQueue = (mode != ClientToServerForQueue);
+    if (notForQueue) {
       int curCnt = this.getClientServerCnxCount();
       if (curCnt >= this.maxConnections) {
         logger.warn(LocalizedMessage.create(
@@ -1482,7 +1492,7 @@ public class AcceptorImpl extends Acceptor implements Runnable, CommBufferPool {
       ServerConnection snap[] = this.allSCList; // avoid volatile read
       this.allSCList = (ServerConnection[]) ArrayUtils.insert(snap, snap.length, serverConn);
     }
-    if (communicationMode != CLIENT_TO_SERVER_FOR_QUEUE) {
+    if (notForQueue) {
       incClientServerCnxCount();
     }
     if (isSelector()) {
@@ -1510,23 +1520,6 @@ public class AcceptorImpl extends Acceptor implements Runnable, CommBufferPool {
     }
   }
 
-  private String getCommunicationMode(byte communicationMode) throws IOException {
-    switch (communicationMode) {
-      default:
-        throw new IOException("Acceptor received unknown communication mode: " + communicationMode);
-      case CLIENT_TO_SERVER:
-        return "client";
-      case GATEWAY_TO_GATEWAY:
-        return "gateway";
-      case MONITOR_TO_SERVER:
-        return "monitor";
-      case CLIENT_TO_SERVER_FOR_QUEUE:
-        return "clientToServerForQueue";
-      case PROTOBUF_CLIENT_SERVER_PROTOCOL:
-        return "Protobuf client";
-    }
-  }
-
   @Override
   public boolean isRunning() {
     return !this.shutdownStarted;

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
index e2612fc..cfdec5a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -71,6 +71,7 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.ClassLoadUtil;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.statistics.DummyStatisticsFactory;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.InternalInstantiator;
@@ -261,7 +262,8 @@ public class CacheClientNotifier {
           LocalizedMessage.create(
               LocalizedStrings.CacheClientNotifier_CACHECLIENTNOTIFIER_CAUGHT_EXCEPTION_ATTEMPTING_TO_CLIENT),
           uve);
-      writeException(dos, Acceptor.UNSUCCESSFUL_SERVER_TO_CLIENT, uve, clientVersion);
+      writeException(dos, CommunicationMode.UnsuccessfulServerToClient.getModeNumber(), uve,
+          clientVersion);
       return;
     }
 
@@ -393,7 +395,8 @@ public class CacheClientNotifier {
       logger.warn(LocalizedMessage.create(
           LocalizedStrings.CacheClientNotifier_AN_EXCEPTION_WAS_THROWN_FOR_CLIENT_0_1,
           new Object[] {proxyID, ""}), ex);
-      writeException(dos, Acceptor.UNSUCCESSFUL_SERVER_TO_CLIENT, ex, clientVersion);
+      writeException(dos, CommunicationMode.UnsuccessfulServerToClient.getModeNumber(), ex,
+          clientVersion);
       return;
     }
 
@@ -426,7 +429,7 @@ public class CacheClientNotifier {
     }
 
     // Determine whether the client is durable or not.
-    byte responseByte = Acceptor.SUCCESSFUL_SERVER_TO_CLIENT;
+    byte responseByte = CommunicationMode.SuccessfulServerToClient.getModeNumber();
     String unsuccessfulMsg = null;
     boolean successful = true;
     boolean clientIsDurable = proxyId.isDurable();
@@ -576,7 +579,7 @@ public class CacheClientNotifier {
     // the marker message. If the client is durable, the message processor
     // is not started until the clientReady message is received.
     if (!clientIsDurable && l_proxy != null
-        && responseByte == Acceptor.SUCCESSFUL_SERVER_TO_CLIENT) {
+        && responseByte == CommunicationMode.SuccessfulServerToClient.getModeNumber()) {
       // The startOrResumeMessageDispatcher tests if the proxy is a primary.
       // If this is a secondary proxy, the dispatcher is not started.
       // The false parameter signifies that a marker message has not already been
@@ -584,7 +587,7 @@ public class CacheClientNotifier {
       l_proxy.startOrResumeMessageDispatcher(false);
     }
 
-    if (responseByte == Acceptor.SUCCESSFUL_SERVER_TO_CLIENT) {
+    if (responseByte == CommunicationMode.SuccessfulServerToClient.getModeNumber()) {
       if (logger.isDebugEnabled()) {
         logger.debug("CacheClientNotifier: Successfully registered {}", l_proxy);
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitor.java
index e877852..226da8a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitor.java
@@ -39,6 +39,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.TXId;
 import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.concurrent.ConcurrentHashSet;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -536,7 +537,7 @@ public class ClientHealthMonitor {
         Iterator connectionsIterator = connections.iterator();
         while (connectionsIterator.hasNext()) {
           ServerConnection sc = (ServerConnection) connectionsIterator.next();
-          if (sc.getCommunicationMode() == Acceptor.GATEWAY_TO_GATEWAY) {
+          if (sc.getCommunicationMode().isWAN()) {
             IncomingGatewayStatus status = new IncomingGatewayStatus(proxyID.getDSMembership(),
                 sc.getSocketAddress(), sc.getSocketPort());
             connectedIncomingGateways.put(proxyID.getDSMembership(), status);

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListener.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListener.java
index 104d88a..9f466c7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListener.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListener.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
+import org.apache.geode.internal.cache.tier.CommunicationMode;
+
 /**
  * A listener which can be registered on {@link AcceptorImpl} in order to receive events about
  * connections created or destroyed for this acceptor.
@@ -28,7 +30,7 @@ public interface ConnectionListener {
    * @param firstConnection true if this is the first connection from this client.
    * @param communicationMode the communication mode of this connection.
    */
-  void connectionOpened(boolean firstConnection, byte communicationMode);
+  void connectionOpened(boolean firstConnection, CommunicationMode communicationMode);
 
   /**
    * Indicates that the a connection to this acceptor has been closed.
@@ -36,7 +38,7 @@ public interface ConnectionListener {
    * @param lastConnection indicates that this was the last connection from this client.
    * @param communicationMode of this connection.
    */
-  void connectionClosed(boolean lastConnection, byte communicationMode);
+  void connectionClosed(boolean lastConnection, CommunicationMode communicationMode);
 
   /**
    * Indicates that a new queue was created on this Acceptor.

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListenerAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListenerAdapter.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListenerAdapter.java
index 7476b4f..3824920 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListenerAdapter.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ConnectionListenerAdapter.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
+import org.apache.geode.internal.cache.tier.CommunicationMode;
+
 /**
  * A ConnectionListener that does nothing.
  * 
@@ -22,9 +24,9 @@ package org.apache.geode.internal.cache.tier.sockets;
  */
 public class ConnectionListenerAdapter implements ConnectionListener {
 
-  public void connectionClosed(boolean lastConnection, byte communicationMode) {}
+  public void connectionClosed(boolean lastConnection, CommunicationMode communicationMode) {}
 
-  public void connectionOpened(boolean firstConnection, byte communicationMode) {}
+  public void connectionOpened(boolean firstConnection, CommunicationMode communicationMode) {}
 
   public void queueAdded(ClientProxyMembershipID id) {}
 

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/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 690fd83..7fa6fe3 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
@@ -44,6 +44,7 @@ import org.apache.geode.internal.VersionedDataInputStream;
 import org.apache.geode.internal.VersionedDataOutputStream;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.ClientHandShake;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.ConnectionProxy;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.InternalLogWriter;
@@ -257,7 +258,7 @@ public class HandShake implements ClientHandShake {
    * HandShake Constructor used by server side connection
    */
   public HandShake(Socket sock, int timeout, DistributedSystem sys, Version clientVersion,
-      byte communicationMode, SecurityService securityService)
+      CommunicationMode communicationMode, SecurityService securityService)
       throws IOException, AuthenticationRequiredException {
 
     this.clientVersion = clientVersion;
@@ -300,8 +301,7 @@ public class HandShake implements ClientHandShake {
             setClientConflation(dis.readByte());
           }
           // Hitesh
-          if (this.clientVersion.compareTo(Version.GFE_65) < 0
-              || communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
+          if (this.clientVersion.compareTo(Version.GFE_65) < 0 || communicationMode.isWAN()) {
             this.credentials = readCredentials(dis, dos, sys, this.securityService);
           } else {
             this.credentials = this.readCredential(dis, dos, sys);
@@ -453,13 +453,13 @@ public class HandShake implements ClientHandShake {
   /**
    * client-to-server handshake. Nothing is sent to the server prior to invoking this method.
    */
-  private byte write(DataOutputStream dos, DataInputStream dis, byte communicationMode,
+  private byte write(DataOutputStream dos, DataInputStream dis, CommunicationMode communicationMode,
       int replyCode, int readTimeout, List ports, Properties p_credentials,
       DistributedMember member, boolean isCallbackConnection) throws IOException {
     HeapDataOutputStream hdos = new HeapDataOutputStream(32, Version.CURRENT);
     byte acceptanceCode = -1;
     try {
-      hdos.writeByte(communicationMode);
+      hdos.writeByte(communicationMode.getModeNumber());
       if (overrideClientVersion > 0) {
         // for testing
         Version.writeOrdinal(hdos, overrideClientVersion, true);
@@ -495,9 +495,8 @@ public class HandShake implements ClientHandShake {
         }
       }
 
-      if (isCallbackConnection || communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
-        if (isCallbackConnection && this.multiuserSecureMode
-            && communicationMode != Acceptor.GATEWAY_TO_GATEWAY) {
+      if (isCallbackConnection || communicationMode.isWAN()) {
+        if (isCallbackConnection && this.multiuserSecureMode && !communicationMode.isWAN()) {
           hdos.writeByte(SECURITY_MULTIUSER_NOTIFICATIONCHANNEL);
           hdos.flush();
           dos.write(hdos.toByteArray());
@@ -1111,7 +1110,7 @@ public class HandShake implements ClientHandShake {
   }
 
   public void accept(OutputStream out, InputStream in, byte epType, int qSize,
-      byte communicationMode, Principal principal) throws IOException {
+      CommunicationMode communicationMode, Principal principal) throws IOException {
     DataOutputStream dos = new DataOutputStream(out);
     DataInputStream dis;
     if (clientVersion.compareTo(Version.CURRENT) < 0) {
@@ -1121,7 +1120,7 @@ public class HandShake implements ClientHandShake {
       dis = new DataInputStream(in);
     }
     // Write ok reply
-    if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY && principal != null) {
+    if (communicationMode.isWAN() && principal != null) {
       dos.writeByte(REPLY_WAN_CREDENTIALS);
     } else {
       dos.writeByte(REPLY_OK);// byte 59
@@ -1129,7 +1128,7 @@ public class HandShake implements ClientHandShake {
 
 
     // additional byte of wan site needs to send for Gateway BC
-    if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
+    if (communicationMode.isWAN()) {
       Version.writeOrdinal(dos, ServerHandShakeProcessor.currentServerVersion.ordinal(), true);
     }
 
@@ -1144,28 +1143,25 @@ public class HandShake implements ClientHandShake {
     dos.writeUTF("");
 
     // Write delta-propagation property value if this is not WAN.
-    if (communicationMode != Acceptor.GATEWAY_TO_GATEWAY
-        && this.clientVersion.compareTo(Version.GFE_61) >= 0) {
+    if (!communicationMode.isWAN() && this.clientVersion.compareTo(Version.GFE_61) >= 0) {
       dos.writeBoolean(((InternalDistributedSystem) this.system).getConfig().getDeltaPropagation());
     }
 
     // Neeraj: Now if the communication mode is GATEWAY_TO_GATEWAY
     // and principal not equal to null then send the credentials also
-    if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY && principal != null) {
+    if (communicationMode.isWAN() && principal != null) {
       sendCredentialsForWan(dos, dis);
     }
 
     // Write the distributed system id if this is a 6.6 or greater client
     // on the remote side of the gateway
-    if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY
-        && this.clientVersion.compareTo(Version.GFE_66) >= 0
+    if (communicationMode.isWAN() && this.clientVersion.compareTo(Version.GFE_66) >= 0
         && ServerHandShakeProcessor.currentServerVersion.compareTo(Version.GFE_66) >= 0) {
       dos.writeByte(((InternalDistributedSystem) this.system).getDistributionManager()
           .getDistributedSystemId());
     }
 
-    if ((communicationMode == Acceptor.GATEWAY_TO_GATEWAY)
-        && this.clientVersion.compareTo(Version.GFE_80) >= 0
+    if ((communicationMode.isWAN()) && this.clientVersion.compareTo(Version.GFE_80) >= 0
         && ServerHandShakeProcessor.currentServerVersion.compareTo(Version.GFE_80) >= 0) {
       int pdxSize = PeerTypeRegistration.getPdxRegistrySize();
       dos.writeInt(pdxSize);
@@ -1189,7 +1185,7 @@ public class HandShake implements ClientHandShake {
    * Client-side handshake with a Server
    */
   public ServerQueueStatus handshakeWithServer(Connection conn, ServerLocation location,
-      byte communicationMode) throws IOException, AuthenticationRequiredException,
+      CommunicationMode communicationMode) throws IOException, AuthenticationRequiredException,
       AuthenticationFailedException, ServerRefusedConnectionException {
     try {
       ServerQueueStatus serverQStatus = null;
@@ -1209,15 +1205,14 @@ public class HandShake implements ClientHandShake {
           updateProxyID(dm.getDistributionManagerId());
         }
       }
-      if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY) {
+      if (communicationMode.isWAN()) {
         this.credentials = getCredentials(member);
       }
       byte intermediateAcceptanceCode = write(dos, dis, communicationMode, REPLY_OK,
           this.clientReadTimeout, null, this.credentials, member, false);
 
       String authInit = this.system.getProperties().getProperty(SECURITY_CLIENT_AUTH_INIT);
-      if (communicationMode != Acceptor.GATEWAY_TO_GATEWAY
-          && intermediateAcceptanceCode != REPLY_AUTH_NOT_REQUIRED
+      if (!communicationMode.isWAN() && intermediateAcceptanceCode != REPLY_AUTH_NOT_REQUIRED
           && (authInit != null && authInit.length() != 0)) {
         location.compareAndSetRequiresCredentials(true);
       }
@@ -1235,9 +1230,8 @@ public class HandShake implements ClientHandShake {
       }
 
       // Successful handshake for GATEWAY_TO_GATEWAY mode sets the peer version in connection
-      if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY
-          && !(acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_REQUIRED
-              || acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
+      if (communicationMode.isWAN() && !(acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_REQUIRED
+          || acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
         short wanSiteVersion = Version.readOrdinal(dis);
         conn.setWanSiteVersion(wanSiteVersion);
         // establish a versioned stream for the other site, if necessary
@@ -1261,14 +1255,12 @@ public class HandShake implements ClientHandShake {
       // Read delta-propagation property value from server.
       // [sumedh] Static variable below? Client can connect to different
       // DSes with different values of this. It shoule be a member variable.
-      if (communicationMode != Acceptor.GATEWAY_TO_GATEWAY
-          && currentClientVersion.compareTo(Version.GFE_61) >= 0) {
+      if (!communicationMode.isWAN() && currentClientVersion.compareTo(Version.GFE_61) >= 0) {
         deltaEnabledOnServer = dis.readBoolean();
       }
 
       // validate that the remote side has a different distributed system id.
-      if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY
-          && Version.GFE_66.compareTo(conn.getWanSiteVersion()) <= 0
+      if (communicationMode.isWAN() && Version.GFE_66.compareTo(conn.getWanSiteVersion()) <= 0
           && currentClientVersion.compareTo(Version.GFE_66) >= 0) {
         int remoteDistributedSystemId = in.read();
         int localDistributedSystemId =
@@ -1281,8 +1273,7 @@ public class HandShake implements ClientHandShake {
         }
       }
       // Read the PDX registry size from the remote size
-      if (communicationMode == Acceptor.GATEWAY_TO_GATEWAY
-          && Version.GFE_80.compareTo(conn.getWanSiteVersion()) <= 0
+      if (communicationMode.isWAN() && Version.GFE_80.compareTo(conn.getWanSiteVersion()) <= 0
           && currentClientVersion.compareTo(Version.GFE_80) >= 0) {
         int remotePdxSize = dis.readInt();
         serverQStatus.setPdxSize(remotePdxSize);
@@ -1312,8 +1303,8 @@ public class HandShake implements ClientHandShake {
       if (!this.multiuserSecureMode) {
         this.credentials = getCredentials(member);
       }
-      byte mode =
-          isPrimary ? Acceptor.PRIMARY_SERVER_TO_CLIENT : Acceptor.SECONDARY_SERVER_TO_CLIENT;
+      CommunicationMode mode = isPrimary ? CommunicationMode.PrimaryServerToClient
+          : CommunicationMode.SecondaryServerToClient;
       write(dos, dis, mode, REPLY_OK, 0, new ArrayList(), this.credentials, member, true);
 
       // Wait here for a reply before continuing. This ensures that the client

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/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 1925898..b243d8e 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
@@ -52,6 +52,7 @@ import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.ClientHandShake;
 import org.apache.geode.internal.cache.tier.Command;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.InternalClientMembership;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.command.Default;
@@ -173,7 +174,7 @@ public abstract class ServerConnection implements Runnable {
 
   private final InternalLogWriter logWriter;
   private final InternalLogWriter securityLogWriter;
-  final private AcceptorImpl acceptor;
+  final AcceptorImpl acceptor;
   private Thread owner;
 
   /**
@@ -224,7 +225,7 @@ public abstract class ServerConnection implements Runnable {
    * The communication mode for this <code>ServerConnection</code>. Valid types include
    * 'client-server', 'gateway-gateway' and 'monitor-server'.
    */
-  protected final byte communicationMode;
+  protected final CommunicationMode communicationMode;
   private final String communicationModeStr;
 
   private long processingMessageStartTime = -1;
@@ -280,7 +281,7 @@ public abstract class ServerConnection implements Runnable {
     this.logWriter = (InternalLogWriter) internalCache.getLoggerI18n();
     this.securityLogWriter = (InternalLogWriter) internalCache.getSecurityLoggerI18n();
     this.communicationModeStr = communicationModeStr;
-    this.communicationMode = communicationMode;
+    this.communicationMode = CommunicationMode.fromModeNumber(communicationMode);
     this.principal = null;
     this.authzRequest = null;
     this.postAuthzRequest = null;
@@ -615,7 +616,7 @@ public abstract class ServerConnection implements Runnable {
       logger.debug("{}: Accepted handshake", this.name);
     }
 
-    if (this.communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE) {
+    if (this.communicationMode == CommunicationMode.ClientToServerForQueue) {
       this.stats.incCurrentQueueConnections();
     } else {
       this.stats.incCurrentClientConnections();
@@ -680,10 +681,7 @@ public abstract class ServerConnection implements Runnable {
    * @return whether this is a connection to a client, regardless of protocol.
    */
   public boolean isClientServerConnection() {
-    return communicationMode == Acceptor.CLIENT_TO_SERVER
-        || communicationMode == Acceptor.PRIMARY_SERVER_TO_CLIENT
-        || communicationMode == Acceptor.SECONDARY_SERVER_TO_CLIENT
-        || communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE;
+    return communicationMode.isClientToServerOrSubscriptionFeed();
   }
 
   static class Counter {
@@ -714,7 +712,7 @@ public abstract class ServerConnection implements Runnable {
       // can be used.
       initializeCommands();
       // its initialized in verifyClientConnection call
-      if (getCommunicationMode() != Acceptor.GATEWAY_TO_GATEWAY)
+      if (!getCommunicationMode().isWAN())
         initializeClientUserAuths();
     }
     if (TEST_VERSION_AFTER_HANDSHAKE_FLAG) {
@@ -777,7 +775,7 @@ public abstract class ServerConnection implements Runnable {
         // authorization later
         if (AcceptorImpl.isIntegratedSecurity()
             && !isInternalMessage(this.requestMsg, ALLOW_INTERNAL_MESSAGES_WITHOUT_CREDENTIALS)
-            && this.communicationMode != Acceptor.GATEWAY_TO_GATEWAY) {
+            && !this.communicationMode.isWAN()) {
           long uniqueId = getUniqueId();
           Subject subject = this.clientUserAuths.getSubject(uniqueId);
           if (subject != null) {
@@ -846,7 +844,7 @@ public abstract class ServerConnection implements Runnable {
             this.stats.decCurrentClients();
           }
         }
-        if (this.communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE) {
+        if (this.communicationMode == CommunicationMode.ClientToServerForQueue) {
           this.stats.decCurrentQueueConnections();
         } else {
           this.stats.decCurrentClientConnections();
@@ -1079,9 +1077,8 @@ public abstract class ServerConnection implements Runnable {
     // need to take care all message types here
     if (AcceptorImpl.isAuthenticationRequired()
         && this.handshake.getVersion().compareTo(Version.GFE_65) >= 0
-        && (this.communicationMode != Acceptor.GATEWAY_TO_GATEWAY)
-        && (!this.requestMsg.getAndResetIsMetaRegion())
-        && (!isInternalMessage(this.requestMsg, ALLOW_INTERNAL_MESSAGES_WITHOUT_CREDENTIALS))) {
+        && !this.communicationMode.isWAN() && !this.requestMsg.getAndResetIsMetaRegion()
+        && !isInternalMessage(this.requestMsg, ALLOW_INTERNAL_MESSAGES_WITHOUT_CREDENTIALS)) {
       setSecurityPart();
       return this.securePart;
     } else {
@@ -1248,7 +1245,7 @@ public abstract class ServerConnection implements Runnable {
   // return this.proxyId.getDistributedMember();
   // }
 
-  protected byte getCommunicationMode() {
+  protected CommunicationMode getCommunicationMode() {
     return this.communicationMode;
   }
 
@@ -1446,10 +1443,8 @@ public abstract class ServerConnection implements Runnable {
     if (isClosed()) {
       return false;
     }
-    if (this.communicationMode == Acceptor.CLIENT_TO_SERVER || isGatewayConnection()
-        || this.communicationMode == Acceptor.MONITOR_TO_SERVER
-        || this.communicationMode == Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL
-    /* || this.communicationMode == Acceptor.CLIENT_TO_SERVER_FOR_QUEUE */) {
+    if (this.communicationMode.isWAN()
+        || this.communicationMode.isCountedAsClientServerConnection()) {
       getAcceptor().decClientServerCnxCount();
     }
     try {
@@ -1725,7 +1720,7 @@ public abstract class ServerConnection implements Runnable {
   public long getUniqueId() {
     long uniqueId = 0;
 
-    if (this.handshake.getVersion().isPre65() || isGatewayConnection()) {
+    if (this.handshake.getVersion().isPre65() || communicationMode.isWAN()) {
       uniqueId = this.userAuthId;
     } else if (this.requestMsg.isSecureMode()) {
       uniqueId = messageIdExtractor.getUniqueIdFromMessage(this.requestMsg,
@@ -1737,10 +1732,6 @@ public abstract class ServerConnection implements Runnable {
     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

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
index 00e8b88..f0348c3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
@@ -15,6 +15,8 @@
 
 package org.apache.geode.internal.cache.tier.sockets;
 
+import static org.apache.geode.internal.cache.tier.CommunicationMode.ProtobufClientServerProtocol;
+
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
@@ -87,9 +89,9 @@ public class ServerConnectionFactory {
       CacheServerStats stats, int hsTimeout, int socketBufferSize, String communicationModeStr,
       byte communicationMode, Acceptor acceptor, SecurityService securityService)
       throws IOException {
-    if (communicationMode == Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL) {
+    if (communicationMode == ProtobufClientServerProtocol.getModeNumber()) {
       if (!Boolean.getBoolean("geode.feature-protobuf-protocol")) {
-        throw new IOException("Acceptor received unknown communication mode: " + communicationMode);
+        throw new IOException("Server received unknown communication mode: " + communicationMode);
       } else {
         String authenticationMode =
             System.getProperty("geode.protocol-authentication-mode", "NOOP");

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerHandShakeProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerHandShakeProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerHandShakeProcessor.java
index 47e6f3d..9c0bd48 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerHandShakeProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerHandShakeProcessor.java
@@ -29,6 +29,7 @@ import java.net.SocketTimeoutException;
 import java.security.Principal;
 import java.util.Properties;
 
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.logging.log4j.Logger;
 import org.apache.shiro.subject.Subject;
@@ -214,7 +215,7 @@ public class ServerHandShakeProcessor {
       // hitesh: it gets principals
       // Hitesh:for older version we should set this
       if (clientVersion.compareTo(Version.GFE_65) < 0
-          || connection.getCommunicationMode() == Acceptor.GATEWAY_TO_GATEWAY) {
+          || connection.getCommunicationMode().isWAN()) {
         long uniqueId = setAuthAttributes(connection);
         connection.setUserAuthId(uniqueId);// for older clients < 6.5
       }
@@ -374,7 +375,7 @@ public class ServerHandShakeProcessor {
         clientVersion = Version.fromOrdinal(clientVersionOrdinal, true);
       } catch (UnsupportedVersionException uve) {
         // Allows higher version of wan site to connect to server
-        if (connection.getCommunicationMode() == Acceptor.GATEWAY_TO_GATEWAY) {
+        if (connection.getCommunicationMode().isWAN()) {
           return Acceptor.VERSION;
         } else {
           SocketAddress sa = socket.getRemoteSocketAddress();

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/test/java/org/apache/geode/cache/server/internal/LoadMonitorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/server/internal/LoadMonitorTest.java b/geode-core/src/test/java/org/apache/geode/cache/server/internal/LoadMonitorTest.java
new file mode 100644
index 0000000..9bff8b7
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/server/internal/LoadMonitorTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cache.server.internal;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.server.ServerLoadProbe;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LoadMonitorTest {
+
+  @Test
+  public void protobufConnectionIsIncludedInLoadMetrics() throws Exception {
+    ServerLoadProbe probe = mock(ServerLoadProbe.class);
+    when(probe.getLoad(any())).thenReturn(null);
+    LoadMonitor loadMonitor = new LoadMonitor(probe, 10000, 0, 0, null);
+    loadMonitor.connectionOpened(true, CommunicationMode.ProtobufClientServerProtocol);
+    assertEquals(1, loadMonitor.metrics.getClientCount());
+    assertEquals(1, loadMonitor.metrics.getConnectionCount());
+  }
+
+  @Test
+  public void protobufConnectionIsRemovedFromLoadMetrics() throws Exception {
+    ServerLoadProbe probe = mock(ServerLoadProbe.class);
+    when(probe.getLoad(any())).thenReturn(null);
+    LoadMonitor loadMonitor = new LoadMonitor(probe, 10000, 0, 0, null);
+    loadMonitor.connectionOpened(true, CommunicationMode.ProtobufClientServerProtocol);
+    loadMonitor.connectionClosed(true, CommunicationMode.ProtobufClientServerProtocol);
+    assertEquals(0, loadMonitor.metrics.getClientCount());
+    assertEquals(0, loadMonitor.metrics.getConnectionCount());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
index ea00018..f74def9 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
@@ -21,8 +21,8 @@ import static org.mockito.Mockito.when;
 
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.security.server.NoOpAuthenticator;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -57,7 +57,7 @@ public class GenericProtocolServerConnectionTest {
 
     return new GenericProtocolServerConnection(socketMock, mock(InternalCache.class),
         mock(CachedRegionHelper.class), mock(CacheServerStats.class), 0, 0, "",
-        Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL, mock(AcceptorImpl.class), clientProtocolMock,
-        mock(SecurityService.class), new NoOpAuthenticator());
+        CommunicationMode.ProtobufClientServerProtocol.getModeNumber(), mock(AcceptorImpl.class),
+        clientProtocolMock, mock(SecurityService.class), new NoOpAuthenticator());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactoryTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactoryTest.java
index cffa05f..09c5949 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactoryTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactoryTest.java
@@ -16,7 +16,7 @@
 package org.apache.geode.internal.cache.tier.sockets;
 
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -53,8 +53,8 @@ public class ServerConnectionFactoryTest {
    */
   @Test(expected = IOException.class)
   public void newClientProtocolFailsWithoutSystemPropertySet() throws IOException {
-    ServerConnection serverConnection =
-        serverConnectionMockedExceptForCommunicationMode(Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL);
+    ServerConnection serverConnection = serverConnectionMockedExceptForCommunicationMode(
+        CommunicationMode.ProtobufClientServerProtocol.getModeNumber());
 
   }
 
@@ -65,21 +65,21 @@ public class ServerConnectionFactoryTest {
   @Test(expected = ServiceLoadingFailureException.class)
   public void newClientProtocolFailsWithSystemPropertySet() throws IOException {
     System.setProperty("geode.feature-protobuf-protocol", "true");
-    ServerConnection serverConnection =
-        serverConnectionMockedExceptForCommunicationMode(Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL);
+    ServerConnection serverConnection = serverConnectionMockedExceptForCommunicationMode(
+        CommunicationMode.ProtobufClientServerProtocol.getModeNumber());
   }
 
   @Test
   public void makeServerConnection() throws Exception {
-    byte[] communicationModes =
-        new byte[] {Acceptor.CLIENT_TO_SERVER, Acceptor.PRIMARY_SERVER_TO_CLIENT,
-            Acceptor.SECONDARY_SERVER_TO_CLIENT, Acceptor.GATEWAY_TO_GATEWAY,
-            Acceptor.MONITOR_TO_SERVER, Acceptor.SUCCESSFUL_SERVER_TO_CLIENT,
-            Acceptor.UNSUCCESSFUL_SERVER_TO_CLIENT, Acceptor.CLIENT_TO_SERVER_FOR_QUEUE,};
+    CommunicationMode[] communicationModes = new CommunicationMode[] {
+        CommunicationMode.ClientToServer, CommunicationMode.PrimaryServerToClient,
+        CommunicationMode.SecondaryServerToClient, CommunicationMode.GatewayToGateway,
+        CommunicationMode.MonitorToServer, CommunicationMode.SuccessfulServerToClient,
+        CommunicationMode.UnsuccessfulServerToClient, CommunicationMode.ClientToServer,};
 
-    for (byte communicationMode : communicationModes) {
+    for (CommunicationMode communicationMode : communicationModes) {
       ServerConnection serverConnection =
-          serverConnectionMockedExceptForCommunicationMode(communicationMode);
+          serverConnectionMockedExceptForCommunicationMode(communicationMode.getModeNumber());
       assertTrue(serverConnection instanceof LegacyServerConnection);
     }
   }
@@ -87,15 +87,15 @@ public class ServerConnectionFactoryTest {
   @Test
   public void makeServerConnectionForOldProtocolWithFeatureFlagEnabled() throws IOException {
     System.setProperty("geode.feature-protobuf-protocol", "true");
-    byte[] communicationModes =
-        new byte[] {Acceptor.CLIENT_TO_SERVER, Acceptor.PRIMARY_SERVER_TO_CLIENT,
-            Acceptor.SECONDARY_SERVER_TO_CLIENT, Acceptor.GATEWAY_TO_GATEWAY,
-            Acceptor.MONITOR_TO_SERVER, Acceptor.SUCCESSFUL_SERVER_TO_CLIENT,
-            Acceptor.UNSUCCESSFUL_SERVER_TO_CLIENT, Acceptor.CLIENT_TO_SERVER_FOR_QUEUE,};
+    CommunicationMode[] communicationModes = new CommunicationMode[] {
+        CommunicationMode.ClientToServer, CommunicationMode.PrimaryServerToClient,
+        CommunicationMode.SecondaryServerToClient, CommunicationMode.GatewayToGateway,
+        CommunicationMode.MonitorToServer, CommunicationMode.SuccessfulServerToClient,
+        CommunicationMode.UnsuccessfulServerToClient, CommunicationMode.ClientToServer,};
 
-    for (byte communicationMode : communicationModes) {
+    for (CommunicationMode communicationMode : communicationModes) {
       ServerConnection serverConnection =
-          serverConnectionMockedExceptForCommunicationMode(communicationMode);
+          serverConnectionMockedExceptForCommunicationMode(communicationMode.getModeNumber());
       assertTrue(serverConnection instanceof LegacyServerConnection);
     }
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/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
index 2aa8995..bd23223 100644
--- 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
@@ -25,11 +25,10 @@ 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.i18n.StringId;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.security.AuthenticationRequiredException;
 import org.apache.geode.test.junit.categories.UnitTest;
@@ -83,8 +82,9 @@ public class ServerConnectionTest {
     InternalCache cache = mock(InternalCache.class);
     SecurityService securityService = mock(SecurityService.class);
 
-    serverConnection = new ServerConnectionFactory().makeServerConnection(socket, cache, null, null,
-        0, 0, null, Acceptor.PRIMARY_SERVER_TO_CLIENT, acceptor, securityService);
+    serverConnection =
+        new ServerConnectionFactory().makeServerConnection(socket, cache, null, null, 0, 0, null,
+            CommunicationMode.PrimaryServerToClient.getModeNumber(), acceptor, securityService);
     MockitoAnnotations.initMocks(this);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
index fbd582a..a4054d4 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
@@ -2,6 +2,7 @@ org/apache/geode/cache/query/CqAttributesFactory
 org/apache/geode/cache/query/internal/index/CompactRangeIndex
 org/apache/geode/cache/query/internal/index/CompactRangeIndex$1
 org/apache/geode/cache/query/internal/index/CompactRangeIndex$IMQEvaluator$1
+org/apache/geode/internal/cache/tier/CommunicationMode
 org/apache/geode/distributed/LocatorLauncher
 org/apache/geode/distributed/LocatorLauncher$Command
 org/apache/geode/distributed/ServerLauncher

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthenticationIntegrationTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthenticationIntegrationTest.java
index 122b8e3..02a053f 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthenticationIntegrationTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthenticationIntegrationTest.java
@@ -28,6 +28,7 @@ import org.apache.geode.security.SecurityManager;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.awaitility.Awaitility;
+import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
@@ -52,6 +53,7 @@ public class AuthenticationIntegrationTest {
 
   private static final String TEST_USERNAME = "bob";
   private static final String TEST_PASSWORD = "bobspassword";
+  private Cache cache;
 
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
@@ -79,7 +81,7 @@ public class AuthenticationIntegrationTest {
     cacheFactory.set(ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION, "false");
 
     cacheFactory.setSecurityManager(mockSecurityManager);
-    Cache cache = cacheFactory.create();
+    cache = cacheFactory.create();
 
     CacheServer cacheServer = cache.addCacheServer();
     int cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();
@@ -99,6 +101,14 @@ public class AuthenticationIntegrationTest {
     protobufProtocolSerializer = new ProtobufProtocolSerializer();
   }
 
+  @After
+  public void tearDown() {
+    if (cache != null) {
+      cache.close();
+      cache = null;
+    }
+  }
+
   @Test
   public void noopAuthenticationSucceeds() throws Exception {
     setUp("NOOP");

http://git-wip-us.apache.org/repos/asf/geode/blob/fecec42f/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripLocatorConnectionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripLocatorConnectionJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripLocatorConnectionJUnitTest.java
index 7ee307b..4457c60 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripLocatorConnectionJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripLocatorConnectionJUnitTest.java
@@ -15,11 +15,11 @@
 
 package org.apache.geode.protocol;
 
+import static org.apache.geode.internal.cache.tier.CommunicationMode.ProtobufClientServerProtocol;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
 import org.apache.geode.protocol.exception.InvalidProtocolMessageException;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.protocol.protobuf.ProtocolErrorCode;
@@ -61,7 +61,7 @@ public class RoundTripLocatorConnectionJUnitTest extends JUnit4CacheTestCase {
     DataOutputStream dataOutputStream = new DataOutputStream(socket.getOutputStream());
     dataOutputStream.writeInt(0);
     // Using the constant from AcceptorImpl to ensure that magic byte is the same
-    dataOutputStream.writeByte(AcceptorImpl.PROTOBUF_CLIENT_SERVER_PROTOCOL);
+    dataOutputStream.writeByte(ProtobufClientServerProtocol.getModeNumber());
   }
 
   @Test