You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2017/10/23 23:36:26 UTC

[geode] 04/04: GEODE-3705: Completed AuthenticationProcessor refactor

This is an automated email from the ASF dual-hosted git repository.

udo pushed a commit to branch feature/GEODE-3705
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 9f7d68ee4304dae1ef44ba5635cbc91d72723bbb
Author: kohlmu-pivotal <uk...@pivotal.io>
AuthorDate: Mon Oct 23 16:33:29 2017 -0700

    GEODE-3705: Completed AuthenticationProcessor refactor
---
 .../distributed/internal/InternalLocator.java      |   1 -
 .../distributed/internal/tcpserver/TcpServer.java  |   6 +-
 .../protocol}/ClientProtocolProcessor.java         |   2 +-
 .../protocol}/ClientProtocolService.java           |   4 +-
 .../protocol}/ClientProtocolServiceLoader.java     |   4 +-
 .../sockets/GenericProtocolServerConnection.java   |   1 +
 .../tier/sockets/ServerConnectionFactory.java      |   3 +
 .../cache/tier/sockets/TcpServerFactory.java       |   1 +
 .../ClientProtocolMessageHandler.java              |  18 +--
 .../apache/geode/internal/protocol/Failure.java    |  17 +--
 .../apache/geode/internal/protocol/Handshaker.java |  63 -----------
 .../MessageExecutionContext.java                   |  24 ++--
 .../geode/internal/protocol/OperationContext.java  |   4 +-
 .../geode/internal/protocol/ProtocolErrorCode.java |   1 -
 .../org/apache/geode/internal/protocol/Result.java |   4 +-
 .../apache/geode/internal/protocol/Success.java    |  18 +--
 .../handshaker/ClientProtocolHandshaker.java       |  29 -----
 .../protocol/operations/OperationHandler.java      |   4 +-
 .../{ => protobuf}/ProtobufCachePipeline.java      |  25 ++---
 .../{ => protobuf}/ProtobufLocatorPipeline.java    |   7 +-
 .../protocol/protobuf/ProtobufOpsProcessor.java    |  45 ++++----
 .../{ => protobuf}/ProtobufProtocolService.java    |  25 ++---
 .../protocol/protobuf/ProtobufStreamProcessor.java |  29 ++---
 .../operations/GetAllRequestOperationHandler.java  |   8 +-
 .../GetAvailableServersOperationHandler.java       |   6 +-
 .../GetRegionNamesRequestOperationHandler.java     |   6 +-
 .../GetRegionRequestOperationHandler.java          |   8 +-
 .../operations/GetRequestOperationHandler.java     |  17 ++-
 .../operations/PutAllRequestOperationHandler.java  |   8 +-
 .../operations/PutRequestOperationHandler.java     |  12 +-
 .../operations/RemoveRequestOperationHandler.java  |  12 +-
 .../HandshakerRequestOperationHandler.java         |  54 ---------
 .../AuthenticationRequestOperationHandler.java     |  45 +++++---
 .../ProtobufOperationContextRegistry.java}         |   6 +-
 .../security/InvalidConfigAuthenticator.java       |   3 +-
 .../security/ProtobufShiroAuthenticator.java       |  13 +--
 .../protobuf/security/ProtobufShiroAuthorizer.java |   2 -
 ...mpatibleAuthenticationMechanismsException.java} |  30 ++---
 .../AuthenticationSecurityProcessor.java           |  10 +-
 .../security/AuthenticationLookupService.java      |  35 ------
 .../security/AuthorizationLookupService.java       |  40 -------
 .../protocol/security/SecurityLookupService.java   |  97 ++++++++++++++++
 .../protocol/security/SecurityProcessor.java       |   7 +-
 .../processors/AuthorizationSecurityProcessor.java |  13 +--
 .../NoAuthenticationSecurityProcessor.java         |  15 ++-
 ...nal.cache.client.protocol.ClientProtocolService |   1 +
 ...ternal.cache.tier.sockets.ClientProtocolService |   1 -
 .../GenericProtocolServerConnectionTest.java       |   3 +-
 .../protocol/AuthenticationIntegrationTest.java    |   8 +-
 .../ProtobufTestExecutionContext.java              |  16 ++-
 .../internal/protocol/protobuf/HandshakerTest.java | 124 ---------------------
 .../ProtobufProtocolServiceJUnitTest.java          |   3 +-
 .../protobuf/ProtobufStreamProcessorTest.java      |  11 +-
 .../protocol/protobuf/ProtobufTestUtilities.java   |  60 ----------
 .../GetAllRequestOperationHandlerJUnitTest.java    |  19 ++--
 ...tAvailableServersOperationHandlerJUnitTest.java |  20 ++--
 ...egionNamesRequestOperationHandlerJUnitTest.java |  26 +++--
 .../GetRegionRequestOperationHandlerJUnitTest.java |  12 +-
 .../GetRequestOperationHandlerJUnitTest.java       |  23 ++--
 .../PutAllRequestOperationHandlerJUnitTest.java    |  15 ++-
 .../PutRequestOperationHandlerJUnitTest.java       |  22 ++--
 .../RemoveRequestOperationHandlerJUnitTest.java    |  19 ++--
 62 files changed, 415 insertions(+), 750 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 9b0ad22..06603cc 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -62,7 +62,6 @@ import org.apache.geode.distributed.internal.tcpserver.TcpServer;
 import org.apache.geode.internal.admin.remote.DistributionLocatorId;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolService;
 import org.apache.geode.internal.cache.tier.sockets.TcpServerFactory;
 import org.apache.geode.internal.cache.wan.WANServiceProvider;
 import org.apache.geode.internal.i18n.LocalizedStrings;
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 34a34d6..a673c03 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
@@ -59,9 +59,9 @@ 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.CommunicationMode;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolProcessor;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolService;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolServiceLoader;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolService;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolServiceLoader;
 import org.apache.geode.internal.cache.tier.sockets.HandShake;
 import org.apache.geode.internal.cache.tier.sockets.ServiceLoadingFailureException;
 import org.apache.geode.internal.logging.LogService;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolProcessor.java
similarity index 96%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolProcessor.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolProcessor.java
index 30b4baf..e49f16f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolProcessor.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.tier.sockets;
+package org.apache.geode.internal.cache.client.protocol;
 
 import java.io.IOException;
 import java.io.InputStream;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolService.java b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolService.java
similarity index 95%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolService.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolService.java
index 79a33a4..7f50c9f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolService.java
@@ -13,9 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.tier.sockets;
-
-import java.util.Map;
+package org.apache.geode.internal.cache.client.protocol;
 
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.Cache;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolServiceLoader.java b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolServiceLoader.java
similarity index 93%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolServiceLoader.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolServiceLoader.java
index 8b069ad..c7ba6e0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolServiceLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/client/protocol/ClientProtocolServiceLoader.java
@@ -13,12 +13,14 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.tier.sockets;
+package org.apache.geode.internal.cache.client.protocol;
 
 import java.util.LinkedList;
 import java.util.List;
 import java.util.ServiceLoader;
 
+import org.apache.geode.internal.cache.tier.sockets.ServiceLoadingFailureException;
+
 public class ClientProtocolServiceLoader {
   private final List<ClientProtocolService> clientProtocolServices;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnection.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnection.java
index 2671cbe..8063bf0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnection.java
@@ -28,6 +28,7 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.CommunicationMode;
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 5f0757f..a6fc973 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
@@ -22,6 +22,9 @@ import java.net.Socket;
 
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolService;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolServiceLoader;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.security.SecurityService;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java
index 47b26d8..1f16dfd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/TcpServerFactory.java
@@ -25,6 +25,7 @@ import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
 import org.apache.geode.distributed.internal.tcpserver.TcpServer;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolServiceLoader;
 import org.apache.geode.internal.logging.LogService;
 
 public class TcpServerFactory {
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ClientProtocolMessageHandler.java
similarity index 72%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ClientProtocolMessageHandler.java
index c1250e1..559f7e1 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ClientProtocolMessageHandler.java
@@ -13,28 +13,28 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.tier.sockets;
+package org.apache.geode.internal.protocol;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 
-import org.apache.geode.StatisticsFactory;
+import org.apache.geode.internal.cache.tier.sockets.GenericProtocolServerConnection;
+import org.apache.geode.internal.cache.tier.sockets.ServerConnectionFactory;
 
 
 /**
- * This is an interface that other modules can implement to hook into {@link
- * GenericProtocolServerConnection} to handle messages sent to Geode.
+ * This is an interface that other modules can implement to hook into
+ * {@link GenericProtocolServerConnection} to handle messages sent to Geode.
  *
  * Currently, only one {@link ClientProtocolMessageHandler} at a time can be used in a Geode
- * instance. It gets wired into {@link ServerConnectionFactory} to create all instances of {@link
- * GenericProtocolServerConnection}.
+ * instance. It gets wired into {@link ServerConnectionFactory} to create all instances of
+ * {@link GenericProtocolServerConnection}.
  *
  * Implementors of this interface are expected to be able to be used for any number of connections
  * at a time (stateless except for the statistics).
  */
 public interface ClientProtocolMessageHandler {
-  MessageExecutionContext receiveMessage(InputStream inputStream, OutputStream outputStream,
-                                         MessageExecutionContext executionContext)
-      throws IOException;
+  void receiveMessage(InputStream inputStream, OutputStream outputStream,
+      MessageExecutionContext executionContext) throws IOException;
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Failure.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Failure.java
index cfcd46d..7bb22e7 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Failure.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Failure.java
@@ -20,31 +20,24 @@ import org.apache.geode.annotations.Experimental;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 
 @Experimental
-public class Failure<SuccessType, ExecutionContext> implements Result<SuccessType, ExecutionContext> {
+public class Failure<SuccessType> implements Result<SuccessType> {
   private final ClientProtocol.ErrorResponse errorResponse;
-  private final ExecutionContext executionContext;
 
-  public Failure(ClientProtocol.ErrorResponse errorResponse, ExecutionContext executionContext) {
+  public Failure(ClientProtocol.ErrorResponse errorResponse) {
     this.errorResponse = errorResponse;
-    this.executionContext = executionContext;
   }
 
-  public static <T, V> Failure<T, V> of(ClientProtocol.ErrorResponse errorResponse, V context) {
-    return new Failure<>(errorResponse, context);
+  public static <T, V> Failure<T> of(ClientProtocol.ErrorResponse errorResponse) {
+    return new Failure<>(errorResponse);
   }
 
   @Override
   public <T> T map(Function<SuccessType, T> successFunction,
-                   Function<ClientProtocol.ErrorResponse, T> errorFunction) {
+      Function<ClientProtocol.ErrorResponse, T> errorFunction) {
     return errorFunction.apply(errorResponse);
   }
 
   @Override
-  public ExecutionContext getExecutionContext() {
-    return executionContext;
-  }
-
-  @Override
   public SuccessType getMessage() {
     throw new RuntimeException("This is not a Success result");
   }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Handshaker.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Handshaker.java
deleted file mode 100644
index 1a5bb97..0000000
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Handshaker.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.protocol;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.cache.IncompatibleVersionException;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.protocol.Result;
-import org.apache.geode.internal.protocol.handshaker.ClientProtocolHandshaker;
-import org.apache.geode.internal.protocol.protobuf.HandshakeAPI;
-import org.apache.geode.internal.protocol.protobuf.operations.handshaker.HandshakerRequestOperationHandler;
-
-public class Handshaker implements ClientProtocolHandshaker {
-  private static final Logger logger = LogService.getLogger();
-
-  private boolean succesfulHandshake = false;
-  private final HandshakerRequestOperationHandler handshakerRequestOperationHandler;
-
-  public Handshaker() {
-    handshakerRequestOperationHandler = new HandshakerRequestOperationHandler();
-  }
-
-  @Override
-  public void processHandshake(InputStream inputStream, OutputStream outputStream)
-      throws IOException, IncompatibleVersionException {
-    HandshakeAPI.HandshakeRequest handshakeRequest =
-        HandshakeAPI.HandshakeRequest.parseDelimitedFrom(inputStream);
-
-    // At this stage HandshakerRequestOperationHandler is not wired into the
-    // ProtobufOpsStreamProcesser.
-    // Thus passing in null serializationService and executionContext.
-    Result<HandshakeAPI.HandshakeResponse,MessageExecutionContext> result =
-        handshakerRequestOperationHandler.process(null, handshakeRequest, null);
-
-    HandshakeAPI.HandshakeResponse handshakeResponse = result.getMessage();
-    handshakeResponse.writeDelimitedTo(outputStream);
-    succesfulHandshake = handshakeResponse.getOk();
-  }
-
-  @Override
-  public boolean completed() {
-    return succesfulHandshake;
-  }
-}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/MessageExecutionContext.java
similarity index 85%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/MessageExecutionContext.java
index 5c9a5c8..f95704b 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/MessageExecutionContext.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.cache.tier.sockets;
+package org.apache.geode.internal.protocol;
 
 
 import org.apache.geode.annotations.Experimental;
@@ -21,7 +21,8 @@ import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
-import org.apache.geode.internal.protocol.protobuf.security.processors.NoAuthenticationSecurityProcessor;
+import org.apache.geode.internal.protocol.security.processors.AuthorizationSecurityProcessor;
+import org.apache.geode.internal.protocol.security.processors.NoAuthenticationSecurityProcessor;
 import org.apache.geode.internal.protocol.security.Authenticator;
 import org.apache.geode.internal.protocol.security.NoOpAuthenticator;
 import org.apache.geode.internal.protocol.security.SecurityProcessor;
@@ -31,12 +32,12 @@ import org.apache.geode.internal.protocol.security.NoOpAuthorizer;
 
 @Experimental
 public class MessageExecutionContext {
-  private Cache cache;
-  private Locator locator;
+  private final Cache cache;
+  private final Locator locator;
   private final Authorizer authorizer;
-  private final Object authenticatedToken;
+  private Object authenticatedToken;
   private final ProtocolClientStatistics statistics;
-  private final SecurityProcessor securityProcessor;
+  private SecurityProcessor securityProcessor;
   private final Authenticator authenticator;
 
 
@@ -44,6 +45,7 @@ public class MessageExecutionContext {
       Authorizer streamAuthorizer, Object authenticatedToken, ProtocolClientStatistics statistics,
       SecurityProcessor securityProcessor) {
     this.cache = cache;
+    this.locator = null;
     this.authorizer = streamAuthorizer;
     this.authenticatedToken = authenticatedToken;
     this.statistics = statistics;
@@ -53,12 +55,12 @@ public class MessageExecutionContext {
 
   public MessageExecutionContext(InternalLocator locator, ProtocolClientStatistics statistics) {
     this.locator = locator;
+    this.cache = null;
     // set a no-op authorizer until such time as locators implement authentication
     // and authorization checks
     this.authorizer = new NoOpAuthorizer();
     this.authenticator = new NoOpAuthenticator();
     this.statistics = statistics;
-    this.authenticatedToken = new Object();
     this.securityProcessor = new NoAuthenticationSecurityProcessor();
   }
 
@@ -120,4 +122,12 @@ public class MessageExecutionContext {
   public SecurityProcessor getSecurityProcessor() {
     return securityProcessor;
   }
+
+  public void setSecurityProcessor(AuthorizationSecurityProcessor securityProcessor) {
+    this.securityProcessor = securityProcessor;
+  }
+
+  public void setAuthenticationToken(Object authenticationToken) {
+    this.authenticatedToken = authenticationToken;
+  }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/OperationContext.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/OperationContext.java
index d4ca2a0..6e2d636 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/OperationContext.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/OperationContext.java
@@ -37,11 +37,11 @@ public class OperationContext<OperationRequest, OperationResponse> {
     this.operationHandler = operationHandler;
     this.fromRequest = fromRequest;
     this.toResponse = toResponse;
-    this.toErrorResponse = OperationContext::makeErrorBuilder;
+    this.toErrorResponse = this::makeErrorBuilder;
     accessPermissionRequired = permissionRequired;
   }
 
-  public static ClientProtocol.Response.Builder makeErrorBuilder(
+  private ClientProtocol.Response.Builder makeErrorBuilder(
       ClientProtocol.ErrorResponse errorResponse) {
     return ClientProtocol.Response.newBuilder().setErrorResponse(errorResponse);
   }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
index f74f613..c9813c1 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
@@ -22,7 +22,6 @@ public enum ProtocolErrorCode {
   UNSUPPORTED_AUTHENTICATION_MODE(1103),
   AUTHENTICATION_FAILED(1200),
   AUTHORIZATION_FAILED(1201),
-  UNAUTHORIZED_REQUEST(1202),
   LOW_MEMORY(1300),
   DATA_UNREACHABLE(1301),
   OPERATION_TIMEOUT(1302),
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Result.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Result.java
index 69a5f6f..6a1a3a4 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Result.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Result.java
@@ -20,12 +20,10 @@ import org.apache.geode.annotations.Experimental;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 
 @Experimental
-public interface Result<SuccessType,ExecutionContext> {
+public interface Result<SuccessType> {
   <T> T map(Function<SuccessType, T> successFunction,
       Function<ClientProtocol.ErrorResponse, T> errorFunction);
 
-  ExecutionContext getExecutionContext();
-
   SuccessType getMessage();
 
   ClientProtocol.ErrorResponse getErrorMessage();
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Success.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Success.java
index 0a3378e..1c1e6f9 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Success.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/Success.java
@@ -20,32 +20,24 @@ import org.apache.geode.annotations.Experimental;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 
 @Experimental
-public class Success<SuccessType, ExecutionContext>
-    implements Result<SuccessType, ExecutionContext> {
+public class Success<SuccessType> implements Result<SuccessType> {
   private final SuccessType successResponse;
-  private final ExecutionContext executionContext;
 
-  public Success(SuccessType successResponse, ExecutionContext executionContext) {
+  public Success(SuccessType successResponse) {
     this.successResponse = successResponse;
-    this.executionContext = executionContext;
   }
 
-  public static <T,V> Success<T,V> of(T result,V context) {
-    return new Success<>(result,context);
+  public static <T> Success<T> of(T result) {
+    return new Success<>(result);
   }
 
   @Override
   public <T> T map(Function<SuccessType, T> successFunction,
-                   Function<ClientProtocol.ErrorResponse, T> errorFunction) {
+      Function<ClientProtocol.ErrorResponse, T> errorFunction) {
     return successFunction.apply(successResponse);
   }
 
   @Override
-  public ExecutionContext getExecutionContext() {
-    return executionContext;
-  }
-
-  @Override
   public SuccessType getMessage() {
     return successResponse;
   }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/handshaker/ClientProtocolHandshaker.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/handshaker/ClientProtocolHandshaker.java
deleted file mode 100644
index da81325..0000000
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/handshaker/ClientProtocolHandshaker.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.protocol.handshaker;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.geode.cache.IncompatibleVersionException;
-import org.apache.geode.internal.protocol.security.Authenticator;
-
-public interface ClientProtocolHandshaker {
-  void processHandshake(InputStream inputStream, OutputStream outputStream)
-      throws IOException, IncompatibleVersionException;
-
-  boolean completed();
-}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/operations/OperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/operations/OperationHandler.java
index 13ce932..286c525 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/operations/OperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/operations/OperationHandler.java
@@ -15,7 +15,7 @@
 package org.apache.geode.internal.protocol.operations;
 
 import org.apache.geode.annotations.Experimental;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.protobuf.ProtobufOpsProcessor;
 import org.apache.geode.internal.protocol.Result;
@@ -33,7 +33,7 @@ public interface OperationHandler<Req, Resp> {
    * Decode the message, deserialize contained values using the serialization service, do the work
    * indicated on the provided cache, and return a response.
    */
-  Result<Resp,MessageExecutionContext> process(SerializationService serializationService, Req request,
+  Result<Resp> process(SerializationService serializationService, Req request,
       MessageExecutionContext messageExecutionContext) throws InvalidExecutionContextException;
 }
 
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufCachePipeline.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufCachePipeline.java
similarity index 66%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufCachePipeline.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufCachePipeline.java
index a7764df..44829e0 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufCachePipeline.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufCachePipeline.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.protocol;
+package org.apache.geode.internal.protocol.protobuf;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -21,44 +21,35 @@ import java.io.OutputStream;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Cache;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolProcessor;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.protocol.protobuf.ProtobufStreamProcessor;
-import org.apache.geode.internal.protocol.protobuf.security.processors.AuthenticationSecurityProcessor;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.security.Authenticator;
 import org.apache.geode.internal.protocol.security.Authorizer;
+import org.apache.geode.internal.protocol.security.SecurityProcessor;
 import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.AuthenticationFailedException;
 
 
 @Experimental
 public final class ProtobufCachePipeline implements ClientProtocolProcessor {
   private final ProtocolClientStatistics statistics;
-  private final Cache cache;
-  private final Authorizer authorizer;
   private final ProtobufStreamProcessor streamProcessor;
-  private final Authenticator authenticator;
 
-  private MessageExecutionContext messageExecutionContext;
+  private final MessageExecutionContext messageExecutionContext;
 
   ProtobufCachePipeline(ProtobufStreamProcessor protobufStreamProcessor,
       ProtocolClientStatistics statistics, Cache cache, Authenticator authenticator,
-      Authorizer authorizer) {
+      Authorizer authorizer, SecurityProcessor securityProcessor) {
     this.streamProcessor = protobufStreamProcessor;
     this.statistics = statistics;
-    this.cache = cache;
-    this.authenticator = authenticator;
-    this.authorizer = authorizer;
     this.statistics.clientConnected();
     this.messageExecutionContext = new MessageExecutionContext(cache, authenticator, authorizer,
-        null, statistics, new AuthenticationSecurityProcessor());
+        null, statistics, securityProcessor);
   }
 
   @Override
   public void processMessage(InputStream inputStream, OutputStream outputStream)
       throws IOException {
-    messageExecutionContext = streamProcessor.receiveMessage(inputStream, outputStream, messageExecutionContext);
+    streamProcessor.receiveMessage(inputStream, outputStream, messageExecutionContext);
   }
 
   @Override
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufLocatorPipeline.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufLocatorPipeline.java
similarity index 88%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufLocatorPipeline.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufLocatorPipeline.java
index bc0bf6a..12aab64 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufLocatorPipeline.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufLocatorPipeline.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.protocol;
+package org.apache.geode.internal.protocol.protobuf;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -22,9 +22,8 @@ import java.io.OutputStream;
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.IncompatibleVersionException;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolProcessor;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.protocol.protobuf.ProtobufStreamProcessor;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
 
 @Experimental
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufOpsProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufOpsProcessor.java
index 59407d6..5529553 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufOpsProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufOpsProcessor.java
@@ -17,17 +17,18 @@ package org.apache.geode.internal.protocol.protobuf;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.protocol.Failure;
 import org.apache.geode.internal.protocol.OperationContext;
 import org.apache.geode.internal.protocol.Result;
-import org.apache.geode.internal.protocol.Success;
-import org.apache.geode.internal.protocol.registry.OperationContextRegistry;
+import org.apache.geode.internal.protocol.protobuf.registry.ProtobufOperationContextRegistry;
 import org.apache.geode.internal.protocol.protobuf.utilities.ProtobufResponseUtilities;
 import org.apache.geode.internal.protocol.security.SecurityProcessor;
 import org.apache.geode.internal.serialization.SerializationService;
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.security.NotAuthorizedException;
 
 import static org.apache.geode.internal.protocol.ProtocolErrorCode.*;
 
@@ -38,51 +39,51 @@ import static org.apache.geode.internal.protocol.ProtocolErrorCode.*;
 @Experimental
 public class ProtobufOpsProcessor {
 
-  private final OperationContextRegistry operationContextRegistry;
+  private final ProtobufOperationContextRegistry protobufOperationContextRegistry;
   private final SerializationService serializationService;
   private static final Logger logger = LogService.getLogger(ProtobufOpsProcessor.class);
 
   public ProtobufOpsProcessor(SerializationService serializationService,
-                              OperationContextRegistry operationContextRegistry) {
+      ProtobufOperationContextRegistry protobufOperationContextRegistry) {
     this.serializationService = serializationService;
-    this.operationContextRegistry = operationContextRegistry;
+    this.protobufOperationContextRegistry = protobufOperationContextRegistry;
   }
 
-  public Result<ClientProtocol.Response, MessageExecutionContext> process(
-      ClientProtocol.Request request,
+  public ClientProtocol.Response process(ClientProtocol.Request request,
       MessageExecutionContext messageExecutionContext) {
     ClientProtocol.Request.RequestAPICase requestType = request.getRequestAPICase();
     logger.debug("Processing request of type {}", requestType);
-    OperationContext operationContext = operationContextRegistry.getOperationContext(requestType);
+    OperationContext operationContext =
+        protobufOperationContextRegistry.getOperationContext(requestType);
     Result result;
 
     SecurityProcessor securityProcessor = messageExecutionContext.getSecurityProcessor();
-    if (securityProcessor.validateOperation(request, messageExecutionContext, operationContext)) {
+    try {
+      securityProcessor.validateOperation(request, messageExecutionContext, operationContext);
       result = processOperation(request, messageExecutionContext, requestType, operationContext);
-    } else {
-      logger.warn("Received unauthorized request");
-      messageExecutionContext.getStatistics().incAuthorizationViolations();
+    } catch (AuthenticationRequiredException e) {
+      logger.warn(e);
+      result = Failure
+          .of(ProtobufResponseUtilities.makeErrorResponse(AUTHENTICATION_FAILED, e.getMessage()));
+    } catch (NotAuthorizedException e) {
+      logger.warn(e);
       result = Failure.of(ProtobufResponseUtilities.makeErrorResponse(AUTHORIZATION_FAILED,
-          "User isn't authorized for this operation."), messageExecutionContext);
+          "The user is not authorized to complete this operation"));
     }
 
-    ClientProtocol.Response
-        response =
-        ((ClientProtocol.Response.Builder) result.map(operationContext.getToResponse(),
-            operationContext.getToErrorResponse())).build();
-    return Success.of(response, (MessageExecutionContext) result.getExecutionContext());
+    return ((ClientProtocol.Response.Builder) result.map(operationContext.getToResponse(),
+        operationContext.getToErrorResponse())).build();
   }
 
   private Result processOperation(ClientProtocol.Request request, MessageExecutionContext context,
-                                  ClientProtocol.Request.RequestAPICase requestType,
-                                  OperationContext operationContext) {
+      ClientProtocol.Request.RequestAPICase requestType, OperationContext operationContext) {
     try {
       return operationContext.getOperationHandler().process(serializationService,
           operationContext.getFromRequest().apply(request), context);
     } catch (InvalidExecutionContextException exception) {
       logger.error("Invalid execution context found for operation {}", requestType);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(UNSUPPORTED_OPERATION,
-          "Invalid execution context found for operation."), context);
+          "Invalid execution context found for operation."));
     }
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufProtocolService.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolService.java
similarity index 74%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufProtocolService.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolService.java
index 97570db..4d8a100 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/ProtobufProtocolService.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolService.java
@@ -12,30 +12,25 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.protocol;
+package org.apache.geode.internal.protocol.protobuf;
 
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolProcessor;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolService;
-import org.apache.geode.internal.protocol.protobuf.ProtobufStreamProcessor;
-import org.apache.geode.internal.protocol.statistics.NoOpStatistics;
-import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolService;
 import org.apache.geode.internal.protocol.protobuf.statistics.ProtobufClientStatisticsImpl;
-import org.apache.geode.internal.protocol.security.AuthenticationLookupService;
 import org.apache.geode.internal.protocol.security.Authenticator;
-import org.apache.geode.internal.protocol.security.AuthorizationLookupService;
 import org.apache.geode.internal.protocol.security.Authorizer;
+import org.apache.geode.internal.protocol.security.SecurityLookupService;
+import org.apache.geode.internal.protocol.statistics.NoOpStatistics;
+import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
 import org.apache.geode.internal.security.SecurityService;
 
 public class ProtobufProtocolService implements ClientProtocolService {
   private volatile ProtocolClientStatistics statistics;
   private final ProtobufStreamProcessor protobufStreamProcessor = new ProtobufStreamProcessor();
-  private final AuthenticationLookupService authenticationLookupService =
-      new AuthenticationLookupService();
-  private final AuthorizationLookupService authorizationLookupService =
-      new AuthorizationLookupService();
+  private final SecurityLookupService securityLookupService = new SecurityLookupService();
 
   @Override
   public synchronized void initializeStatistics(String statisticsName, StatisticsFactory factory) {
@@ -49,11 +44,11 @@ public class ProtobufProtocolService implements ClientProtocolService {
       SecurityService securityService) {
     assert (statistics != null);
 
-    Authenticator authenticator = authenticationLookupService.getAuthenticator(securityService);
-    Authorizer authorizer = authorizationLookupService.getAuthorizer(securityService);
+    Authenticator authenticator = securityLookupService.lookupAuthenticator(securityService);
+    Authorizer authorizer = securityLookupService.lookupAuthorizer(securityService);
 
     return new ProtobufCachePipeline(protobufStreamProcessor, getStatistics(), cache, authenticator,
-        authorizer);
+        authorizer, securityLookupService.lookupProcessor(securityService));
   }
 
   /**
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessor.java
index 4250e93..7bc4730 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessor.java
@@ -22,12 +22,11 @@ import java.io.OutputStream;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
-import org.apache.geode.internal.cache.tier.sockets.ClientProtocolMessageHandler;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.ClientProtocolMessageHandler;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.exception.InvalidProtocolMessageException;
-import org.apache.geode.internal.protocol.registry.OperationContextRegistry;
+import org.apache.geode.internal.protocol.protobuf.registry.ProtobufOperationContextRegistry;
 import org.apache.geode.internal.protocol.protobuf.serializer.ProtobufProtocolSerializer;
 import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
 import org.apache.geode.internal.protocol.protobuf.utilities.ProtobufUtilities;
@@ -46,23 +45,21 @@ public class ProtobufStreamProcessor implements ClientProtocolMessageHandler {
   public ProtobufStreamProcessor() {
     protobufProtocolSerializer = new ProtobufProtocolSerializer();
     protobufOpsProcessor = new ProtobufOpsProcessor(new ProtobufSerializationService(),
-        new OperationContextRegistry());
+        new ProtobufOperationContextRegistry());
   }
 
   @Override
-  public MessageExecutionContext receiveMessage(InputStream inputStream, OutputStream outputStream,
-                                                MessageExecutionContext executionContext)
-      throws IOException {
+  public void receiveMessage(InputStream inputStream, OutputStream outputStream,
+      MessageExecutionContext executionContext) throws IOException {
     try {
-      return processOneMessage(inputStream, outputStream, executionContext);
+      processOneMessage(inputStream, outputStream, executionContext);
     } catch (InvalidProtocolMessageException e) {
       throw new IOException(e);
     }
   }
 
-  private MessageExecutionContext processOneMessage(InputStream inputStream,
-                                                    OutputStream outputStream,
-                                                    MessageExecutionContext executionContext)
+  private void processOneMessage(InputStream inputStream, OutputStream outputStream,
+      MessageExecutionContext executionContext)
       throws InvalidProtocolMessageException, IOException {
     ClientProtocol.Message message = protobufProtocolSerializer.deserialize(inputStream);
     if (message == null) {
@@ -74,13 +71,9 @@ public class ProtobufStreamProcessor implements ClientProtocolMessageHandler {
     statistics.messageReceived(message.getSerializedSize());
 
     ClientProtocol.Request request = message.getRequest();
-    Result<ClientProtocol.Response, MessageExecutionContext> result =
-        protobufOpsProcessor.process(request, executionContext);
-    ClientProtocol.Message
-        responseMessage =
-        ProtobufUtilities.createProtobufResponse(result.getMessage());
+    ClientProtocol.Response response = protobufOpsProcessor.process(request, executionContext);
+    ClientProtocol.Message responseMessage = ProtobufUtilities.createProtobufResponse(response);
     statistics.messageSent(responseMessage.getSerializedSize());
     protobufProtocolSerializer.serialize(responseMessage, outputStream);
-    return result.getExecutionContext();
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandler.java
index c15a44f..c91c9e1 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandler.java
@@ -25,7 +25,7 @@ import org.apache.geode.cache.CacheLoaderException;
 import org.apache.geode.cache.PartitionedRegionStorageException;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.TimeoutException;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
@@ -49,7 +49,7 @@ public class GetAllRequestOperationHandler
   private static final Logger logger = LogService.getLogger();
 
   @Override
-  public Result<RegionAPI.GetAllResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.GetAllResponse> process(SerializationService serializationService,
       RegionAPI.GetAllRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     String regionName = request.getRegionName();
@@ -57,7 +57,7 @@ public class GetAllRequestOperationHandler
     if (region == null) {
       logger.error("Received GetAll request for non-existing region {}", regionName);
       return Failure
-          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"),messageExecutionContext);
+          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"));
     }
 
     Map<Boolean, List<Object>> resultsCollection = request.getKeyList().stream()
@@ -73,7 +73,7 @@ public class GetAllRequestOperationHandler
       responseBuilder.addFailures((BasicTypes.KeyedError) entry);
     }
 
-    return Success.of(responseBuilder.build(),messageExecutionContext);
+    return Success.of(responseBuilder.build());
   }
 
   private Object processOneMessage(SerializationService serializationService, Region region,
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandler.java
index ec2c917..f15e94c 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandler.java
@@ -21,7 +21,7 @@ import java.util.stream.Collectors;
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
@@ -35,7 +35,7 @@ public class GetAvailableServersOperationHandler implements
     OperationHandler<ServerAPI.GetAvailableServersRequest, ServerAPI.GetAvailableServersResponse> {
 
   @Override
-  public Result<ServerAPI.GetAvailableServersResponse,MessageExecutionContext> process(
+  public Result<ServerAPI.GetAvailableServersResponse> process(
       SerializationService serializationService, ServerAPI.GetAvailableServersRequest request,
       MessageExecutionContext messageExecutionContext) throws InvalidExecutionContextException {
 
@@ -51,7 +51,7 @@ public class GetAvailableServersOperationHandler implements
         .collect(Collectors.toList());
     ServerAPI.GetAvailableServersResponse.Builder builder =
         ServerAPI.GetAvailableServersResponse.newBuilder().addAllServers(servers);
-    return Success.of(builder.build(),messageExecutionContext);
+    return Success.of(builder.build());
   }
 
   private BasicTypes.Server getServerProtobufMessage(ServerLocation serverLocation) {
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandler.java
index c8e6573..7e7cf71 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandler.java
@@ -18,7 +18,7 @@ import java.util.Set;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
@@ -32,10 +32,10 @@ public class GetRegionNamesRequestOperationHandler
     implements OperationHandler<RegionAPI.GetRegionNamesRequest, RegionAPI.GetRegionNamesResponse> {
 
   @Override
-  public Result<RegionAPI.GetRegionNamesResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.GetRegionNamesResponse> process(SerializationService serializationService,
       RegionAPI.GetRegionNamesRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     Set<Region<?, ?>> regions = messageExecutionContext.getCache().rootRegions();
-    return Success.of(ProtobufResponseUtilities.createGetRegionNamesResponse(regions),messageExecutionContext);
+    return Success.of(ProtobufResponseUtilities.createGetRegionNamesResponse(regions));
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandler.java
index 032d72c..f602d6a 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandler.java
@@ -18,7 +18,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
@@ -39,7 +39,7 @@ public class GetRegionRequestOperationHandler
   private static final Logger logger = LogService.getLogger();
 
   @Override
-  public Result<RegionAPI.GetRegionResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.GetRegionResponse> process(SerializationService serializationService,
       RegionAPI.GetRegionRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     String regionName = request.getRegionName();
@@ -48,11 +48,11 @@ public class GetRegionRequestOperationHandler
     if (region == null) {
       logger.error("Received GetRegion request for non-existing region {}", regionName);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND,
-          "No region exists for name: " + regionName),messageExecutionContext);
+          "No region exists for name: " + regionName));
     }
 
     BasicTypes.Region protoRegion = ProtobufUtilities.createRegionMessageFromRegion(region);
 
-    return Success.of(RegionAPI.GetRegionResponse.newBuilder().setRegion(protoRegion).build(),messageExecutionContext);
+    return Success.of(RegionAPI.GetRegionResponse.newBuilder().setRegion(protoRegion).build());
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandler.java
index 0e7c6a2..a67bdc9 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandler.java
@@ -18,7 +18,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
@@ -41,8 +41,7 @@ public class GetRequestOperationHandler
   private static final Logger logger = LogService.getLogger();
 
   @Override
-  public Result<RegionAPI.GetResponse, MessageExecutionContext> process(
-      SerializationService serializationService,
+  public Result<RegionAPI.GetResponse> process(SerializationService serializationService,
       RegionAPI.GetRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     String regionName = request.getRegionName();
@@ -50,8 +49,7 @@ public class GetRequestOperationHandler
     if (region == null) {
       logger.error("Received Get request for non-existing region {}", regionName);
       return Failure
-          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"),
-              messageExecutionContext);
+          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"));
     }
 
     try {
@@ -59,21 +57,20 @@ public class GetRequestOperationHandler
       Object resultValue = region.get(decodedKey);
 
       if (resultValue == null) {
-        return Success.of(RegionAPI.GetResponse.newBuilder().build(), messageExecutionContext);
+        return Success.of(RegionAPI.GetResponse.newBuilder().build());
       }
 
       BasicTypes.EncodedValue encodedValue =
           ProtobufUtilities.createEncodedValue(serializationService, resultValue);
-      return Success.of(RegionAPI.GetResponse.newBuilder().setResult(encodedValue).build(),
-          messageExecutionContext);
+      return Success.of(RegionAPI.GetResponse.newBuilder().setResult(encodedValue).build());
     } catch (UnsupportedEncodingTypeException ex) {
       logger.error("Received Get request with unsupported encoding: {}", ex);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR,
-          "Encoding not supported."), messageExecutionContext);
+          "Encoding not supported."));
     } catch (CodecNotRegisteredForTypeException ex) {
       logger.error("Got codec error when decoding Get request: {}", ex);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR,
-          "Codec error in protobuf deserialization."), messageExecutionContext);
+          "Codec error in protobuf deserialization."));
     }
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandler.java
index 54f9391..b085370 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandler.java
@@ -22,7 +22,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
@@ -45,7 +45,7 @@ public class PutAllRequestOperationHandler
   private static final Logger logger = LogManager.getLogger();
 
   @Override
-  public Result<RegionAPI.PutAllResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.PutAllResponse> process(SerializationService serializationService,
       RegionAPI.PutAllRequest putAllRequest, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     String regionName = putAllRequest.getRegionName();
@@ -54,14 +54,14 @@ public class PutAllRequestOperationHandler
     if (region == null) {
       logger.error("Received PutAll request for non-existing region {}", regionName);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND,
-          "Region passed does not exist: " + regionName),messageExecutionContext);
+          "Region passed does not exist: " + regionName));
     }
 
     RegionAPI.PutAllResponse.Builder builder = RegionAPI.PutAllResponse.newBuilder()
         .addAllFailedKeys(putAllRequest.getEntryList().stream()
             .map((entry) -> singlePut(serializationService, region, entry)).filter(Objects::nonNull)
             .collect(Collectors.toList()));
-    return Success.of(builder.build(),messageExecutionContext);
+    return Success.of(builder.build());
   }
 
   private BasicTypes.KeyedError singlePut(SerializationService serializationService, Region region,
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandler.java
index 719fa85..345f829 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandler.java
@@ -18,7 +18,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
@@ -41,7 +41,7 @@ public class PutRequestOperationHandler
   private static final Logger logger = LogService.getLogger();
 
   @Override
-  public Result<RegionAPI.PutResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.PutResponse> process(SerializationService serializationService,
       RegionAPI.PutRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
     String regionName = request.getRegionName();
@@ -49,7 +49,7 @@ public class PutRequestOperationHandler
     if (region == null) {
       logger.warn("Received Put request for non-existing region: {}", regionName);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND,
-          "Region passed by client did not exist: " + regionName),messageExecutionContext);
+          "Region passed by client did not exist: " + regionName));
     }
 
     try {
@@ -59,16 +59,16 @@ public class PutRequestOperationHandler
       Object decodedKey = ProtobufUtilities.decodeValue(serializationService, entry.getKey());
       try {
         region.put(decodedKey, decodedValue);
-        return Success.of(RegionAPI.PutResponse.newBuilder().build(),messageExecutionContext);
+        return Success.of(RegionAPI.PutResponse.newBuilder().build());
       } catch (ClassCastException ex) {
         logger.error("Received Put request with invalid key type: {}", ex);
         return Failure.of(ProtobufResponseUtilities.makeErrorResponse(CONSTRAINT_VIOLATION,
-            "invalid key or value type for region " + regionName),messageExecutionContext);
+            "invalid key or value type for region " + regionName));
       }
     } catch (UnsupportedEncodingTypeException | CodecNotRegisteredForTypeException ex) {
       logger.error("Got codec error when decoding Put request: {}", ex);
       return Failure
-          .of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR, ex.getMessage()),messageExecutionContext);
+          .of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR, ex.getMessage()));
     }
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandler.java
index 2d401fa..2278f78 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandler.java
@@ -19,7 +19,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.Failure;
@@ -40,7 +40,7 @@ public class RemoveRequestOperationHandler
   private static final Logger logger = LogManager.getLogger();
 
   @Override
-  public Result<RegionAPI.RemoveResponse,MessageExecutionContext> process(SerializationService serializationService,
+  public Result<RegionAPI.RemoveResponse> process(SerializationService serializationService,
       RegionAPI.RemoveRequest request, MessageExecutionContext messageExecutionContext)
       throws InvalidExecutionContextException {
 
@@ -49,23 +49,23 @@ public class RemoveRequestOperationHandler
     if (region == null) {
       logger.error("Received Remove request for non-existing region {}", regionName);
       return Failure
-          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"),messageExecutionContext);
+          .of(ProtobufResponseUtilities.makeErrorResponse(REGION_NOT_FOUND, "Region not found"));
     }
 
     try {
       Object decodedKey = ProtobufUtilities.decodeValue(serializationService, request.getKey());
       region.remove(decodedKey);
 
-      return Success.of(RegionAPI.RemoveResponse.newBuilder().build(),messageExecutionContext);
+      return Success.of(RegionAPI.RemoveResponse.newBuilder().build());
     } catch (UnsupportedEncodingTypeException ex) {
       // can be thrown by encoding or decoding.
       logger.error("Received Remove request with unsupported encoding: {}", ex);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR,
-          "Encoding not supported: " + ex.getMessage()),messageExecutionContext);
+          "Encoding not supported: " + ex.getMessage()));
     } catch (CodecNotRegisteredForTypeException ex) {
       logger.error("Got codec error when decoding Remove request: {}", ex);
       return Failure.of(ProtobufResponseUtilities.makeErrorResponse(VALUE_ENCODING_ERROR,
-          "Codec error in protobuf deserialization: " + ex.getMessage()),messageExecutionContext);
+          "Codec error in protobuf deserialization: " + ex.getMessage()));
     }
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/handshaker/HandshakerRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/handshaker/HandshakerRequestOperationHandler.java
deleted file mode 100644
index 0b2e3b6..0000000
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/handshaker/HandshakerRequestOperationHandler.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.protocol.protobuf.operations.handshaker;
-
-import static org.apache.geode.internal.protocol.ProtocolErrorCode.UNSUPPORTED_VERSION;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.protocol.operations.OperationHandler;
-import org.apache.geode.internal.protocol.protobuf.BasicTypes;
-import org.apache.geode.internal.protocol.protobuf.HandshakeAPI;
-import org.apache.geode.internal.protocol.Result;
-import org.apache.geode.internal.protocol.Success;
-import org.apache.geode.internal.serialization.SerializationService;
-
-public class HandshakerRequestOperationHandler
-    implements OperationHandler<HandshakeAPI.HandshakeRequest, HandshakeAPI.HandshakeResponse> {
-  private static final int MAJOR_VERSION = 1;
-  private static final int MINOR_VERSION = 0;
-  private static final Logger logger =
-      LogService.getLogger(HandshakerRequestOperationHandler.class);
-
-  @Override
-  public Result<HandshakeAPI.HandshakeResponse, MessageExecutionContext> process(
-      SerializationService serializationService,
-      HandshakeAPI.HandshakeRequest request, MessageExecutionContext messageExecutionContext) {
-    HandshakeAPI.Semver version = request.getVersion();
-    if (version.getMajor() != MAJOR_VERSION || version.getMinor() < MINOR_VERSION) {
-      logger.warn("Version mismatch: incompatible version. Supported version is: " + MAJOR_VERSION
-          + "." + MINOR_VERSION);
-      return Success.of(HandshakeAPI.HandshakeResponse.newBuilder().setOk(false)
-          .setError(BasicTypes.Error.newBuilder().setErrorCode(UNSUPPORTED_VERSION.codeValue)
-              .setMessage("Version mismatch: incompatible version. Supported version is: "
-                  + MAJOR_VERSION + "." + MINOR_VERSION))
-          .build(), messageExecutionContext);
-    }
-    return Success.of(HandshakeAPI.HandshakeResponse.newBuilder().setOk(true).build(),
-        messageExecutionContext);
-  }
-}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/security/AuthenticationRequestOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/security/AuthenticationRequestOperationHandler.java
index 947563f..eaaf87f 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/security/AuthenticationRequestOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/operations/security/AuthenticationRequestOperationHandler.java
@@ -16,42 +16,59 @@ package org.apache.geode.internal.protocol.protobuf.operations.security;
 
 import java.util.Properties;
 
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
+import org.apache.geode.internal.protocol.Failure;
+import org.apache.geode.internal.protocol.ProtocolErrorCode;
 import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.Success;
 import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.protobuf.AuthenticationAPI;
-import org.apache.geode.internal.protocol.protobuf.security.processors.AuthorizationSecurityProcessor;
+import org.apache.geode.internal.protocol.protobuf.BasicTypes;
+import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
+import org.apache.geode.internal.protocol.protobuf.security.exception.IncompatibleAuthenticationMechanismsException;
+import org.apache.geode.internal.protocol.security.processors.AuthorizationSecurityProcessor;
 import org.apache.geode.internal.protocol.security.Authenticator;
 import org.apache.geode.internal.serialization.SerializationService;
 import org.apache.geode.security.AuthenticationFailedException;
 
 public class AuthenticationRequestOperationHandler implements
     OperationHandler<AuthenticationAPI.AuthenticationRequest, AuthenticationAPI.AuthenticationResponse> {
+  private static final Logger logger = LogManager.getLogger();
+
   @Override
-  public Result<AuthenticationAPI.AuthenticationResponse,MessageExecutionContext> process(
+  public Result<AuthenticationAPI.AuthenticationResponse> process(
       SerializationService serializationService, AuthenticationAPI.AuthenticationRequest request,
       MessageExecutionContext messageExecutionContext) throws InvalidExecutionContextException {
+
     Authenticator authenticator = messageExecutionContext.getAuthenticator();
     Properties properties = new Properties();
     properties.putAll(request.getCredentialsMap());
 
     try {
       Object authenticationToken = authenticator.authenticate(properties);
-      MessageExecutionContext
-          newMessageExecutionContext =
-          new MessageExecutionContext(messageExecutionContext.getCache(),
-              messageExecutionContext.getAuthenticator(), messageExecutionContext.getAuthorizer(),
-              authenticationToken, messageExecutionContext.getStatistics(),
-              new AuthorizationSecurityProcessor());
+      messageExecutionContext.setSecurityProcessor(new AuthorizationSecurityProcessor());
+      messageExecutionContext.setAuthenticationToken(authenticationToken);
       return Success
-          .of(AuthenticationAPI.AuthenticationResponse.newBuilder().setAuthenticated(true).build(),
-              newMessageExecutionContext);
+          .of(AuthenticationAPI.AuthenticationResponse.newBuilder().setAuthenticated(true).build());
+    } catch (IncompatibleAuthenticationMechanismsException e) {
+      return Failure.of(ClientProtocol.ErrorResponse.newBuilder().setError(
+          buildAndLogError(ProtocolErrorCode.UNSUPPORTED_AUTHENTICATION_MODE, e.getMessage(), e))
+          .build());
     } catch (AuthenticationFailedException e) {
-      return Success
-          .of(AuthenticationAPI.AuthenticationResponse.newBuilder().setAuthenticated(false)
-              .build(), messageExecutionContext);
+      return Success.of(
+          AuthenticationAPI.AuthenticationResponse.newBuilder().setAuthenticated(false).build());
     }
   }
+
+  private BasicTypes.Error buildAndLogError(ProtocolErrorCode errorCode, String message,
+      Exception ex) {
+    logger.warn(message, ex);
+
+    return BasicTypes.Error.newBuilder().setErrorCode(errorCode.codeValue).setMessage(message)
+        .build();
+  }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/registry/OperationContextRegistry.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/registry/ProtobufOperationContextRegistry.java
similarity index 97%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/registry/OperationContextRegistry.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/registry/ProtobufOperationContextRegistry.java
index e18a81b..757dd44 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/registry/OperationContextRegistry.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/registry/ProtobufOperationContextRegistry.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.protocol.registry;
+package org.apache.geode.internal.protocol.protobuf.registry;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -34,10 +34,10 @@ import org.apache.geode.internal.protocol.protobuf.operations.security.Authentic
 import org.apache.geode.security.ResourcePermission;
 
 @Experimental
-public class OperationContextRegistry {
+public class ProtobufOperationContextRegistry {
   private Map<RequestAPICase, OperationContext> operationContexts = new ConcurrentHashMap<>();
 
-  public OperationContextRegistry() {
+  public ProtobufOperationContextRegistry() {
     addContexts();
   }
 
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/InvalidConfigAuthenticator.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/InvalidConfigAuthenticator.java
index 409a075..bed24bb 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/InvalidConfigAuthenticator.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/InvalidConfigAuthenticator.java
@@ -25,6 +25,7 @@ import org.apache.geode.internal.protocol.ProtocolErrorCode;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.protocol.protobuf.security.exception.IncompatibleAuthenticationMechanismsException;
 import org.apache.geode.internal.protocol.security.Authenticator;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.security.AuthenticationFailedException;
@@ -36,7 +37,7 @@ public class InvalidConfigAuthenticator implements Authenticator<Object, Object>
   public Object authenticate(Object object) throws AuthenticationFailedException {
     logger.warn(
         "Attempting to authenticate incoming protobuf message using legacy security implementation. This is not supported. Failing authentication.");
-    throw new AuthenticationFailedException(
+    throw new IncompatibleAuthenticationMechanismsException(
         "Attempting to authenticate incoming protobuf message using legacy security implementation. This is not supported. Failing authentication.");
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthenticator.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthenticator.java
index cdee506..fc95751 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthenticator.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthenticator.java
@@ -14,23 +14,14 @@
  */
 package org.apache.geode.internal.protocol.protobuf.security;
 
-import static org.apache.geode.internal.protocol.ProtocolErrorCode.AUTHENTICATION_FAILED;
+import java.util.Properties;
 
-import org.apache.geode.internal.protocol.protobuf.AuthenticationAPI;
+import org.apache.shiro.subject.Subject;
 
-import org.apache.geode.internal.protocol.protobuf.BasicTypes;
-import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.internal.protocol.security.Authenticator;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.security.AuthenticationFailedException;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Properties;
-
-import org.apache.shiro.subject.Subject;
-
 public class ProtobufShiroAuthenticator implements Authenticator<Properties, Subject> {
   private final SecurityService securityService;
 
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthorizer.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthorizer.java
index b078e4b..04149eb 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthorizer.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/ProtobufShiroAuthorizer.java
@@ -36,8 +36,6 @@ public class ProtobufShiroAuthorizer implements Authorizer {
     try {
       securityService.authorize(permissionRequested);
       return true;
-    } catch (NotAuthorizedException ex) {
-      return false;
     } finally {
       threadState.restore();
     }
diff --git a/geode-protobuf/src/main/proto/handshake_API.proto b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/exception/IncompatibleAuthenticationMechanismsException.java
similarity index 63%
rename from geode-protobuf/src/main/proto/handshake_API.proto
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/exception/IncompatibleAuthenticationMechanismsException.java
index b11ae03..1afecb0 100644
--- a/geode-protobuf/src/main/proto/handshake_API.proto
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/exception/IncompatibleAuthenticationMechanismsException.java
@@ -1,7 +1,7 @@
 /*
  * 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
+ * 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
  *
@@ -12,28 +12,12 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.internal.protocol.protobuf.security.exception;
 
-syntax = "proto3";
-package org.apache.geode.internal.protocol.protobuf;
+import org.apache.geode.security.AuthenticationFailedException;
 
-import "basicTypes.proto";
-
-
-enum AuthenticationMode {
-    NONE = 0;
-    SIMPLE = 1;
-}
-
-message HandshakeRequest {
-    Semver version = 1;
+public class IncompatibleAuthenticationMechanismsException extends AuthenticationFailedException {
+  public IncompatibleAuthenticationMechanismsException(String message) {
+    super(message);
+  }
 }
-
-message HandshakeResponse {
-    bool ok = 1;
-    Error error = 2; // only set if not OK.
-}
-
-message Semver{
-    int32 major = 1;
-    int32 minor = 2;
-}
\ No newline at end of file
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthenticationSecurityProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthenticationSecurityProcessor.java
index d164222..0f48c22 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthenticationSecurityProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthenticationSecurityProcessor.java
@@ -14,17 +14,21 @@
  */
 package org.apache.geode.internal.protocol.protobuf.security.processors;
 
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.OperationContext;
 import org.apache.geode.internal.protocol.protobuf.AuthenticationAPI;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.internal.protocol.security.SecurityProcessor;
+import org.apache.geode.security.AuthenticationRequiredException;
 
 public class AuthenticationSecurityProcessor implements SecurityProcessor<ClientProtocol.Request> {
   @Override
-  public boolean validateOperation(ClientProtocol.Request request,
+  public void validateOperation(ClientProtocol.Request request,
       MessageExecutionContext messageExecutionContext, OperationContext operationContext) {
     Object fromRequest = operationContext.getFromRequest().apply(request);
-    return fromRequest instanceof AuthenticationAPI.AuthenticationRequest;
+    if (!(fromRequest instanceof AuthenticationAPI.AuthenticationRequest)) {
+      throw new AuthenticationRequiredException(
+          "Expecting an authentication message. Received a " + fromRequest.getClass() + " message");
+    }
   }
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthenticationLookupService.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthenticationLookupService.java
deleted file mode 100644
index 5d3f04d..0000000
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthenticationLookupService.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.protocol.security;
-
-import org.apache.geode.internal.protocol.protobuf.security.InvalidConfigAuthenticator;
-import org.apache.geode.internal.protocol.protobuf.security.ProtobufShiroAuthenticator;
-import org.apache.geode.internal.security.SecurityService;
-
-public class AuthenticationLookupService {
-  public Authenticator getAuthenticator(SecurityService securityService) {
-    if (securityService.isIntegratedSecurity()) {
-      // Simple authenticator...normal shiro
-      return new ProtobufShiroAuthenticator(securityService);
-    }
-    if (securityService.isPeerSecurityRequired() || securityService.isClientSecurityRequired()) {
-      // Failing authentication...legacy security
-      return new InvalidConfigAuthenticator();
-    } else {
-      // Noop authenticator...no security
-      return new NoOpAuthenticator();
-    }
-  }
-}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthorizationLookupService.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthorizationLookupService.java
deleted file mode 100644
index ed81ea6..0000000
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/AuthorizationLookupService.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.protocol.security;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.ServiceLoader;
-
-import org.apache.geode.GemFireConfigException;
-import org.apache.geode.internal.protocol.protobuf.security.ProtobufShiroAuthorizer;
-import org.apache.geode.internal.security.SecurityService;
-
-public class AuthorizationLookupService {
-  public Authorizer getAuthorizer(SecurityService securityService) {
-    if (securityService.isIntegratedSecurity()) {
-      // Simple authenticator...normal shiro
-      return new ProtobufShiroAuthorizer(securityService);
-    }
-    if (securityService.isPeerSecurityRequired() || securityService.isClientSecurityRequired()) {
-      // Failing authentication...legacy security
-      // This should never be called.
-      return null;
-    } else {
-      // Noop authenticator...no security
-      return new NoOpAuthorizer();
-    }
-  }
-}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityLookupService.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityLookupService.java
new file mode 100644
index 0000000..dc70ec8
--- /dev/null
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityLookupService.java
@@ -0,0 +1,97 @@
+/*
+ * 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.protocol.security;
+
+import org.apache.geode.internal.protocol.protobuf.security.InvalidConfigAuthenticator;
+import org.apache.geode.internal.protocol.protobuf.security.ProtobufShiroAuthenticator;
+import org.apache.geode.internal.protocol.protobuf.security.ProtobufShiroAuthorizer;
+import org.apache.geode.internal.protocol.protobuf.security.processors.AuthenticationSecurityProcessor;
+import org.apache.geode.internal.protocol.security.processors.NoAuthenticationSecurityProcessor;
+import org.apache.geode.internal.security.SecurityService;
+
+public class SecurityLookupService {
+  private final Authenticator[] authenticators = new Authenticator[3];
+  private final Authorizer[] authorizers = new Authorizer[2];
+  private final SecurityProcessor[] securityProcessors = new SecurityProcessor[2];
+
+  public SecurityLookupService() {
+    initializeAuthenticators();
+    initializeAuthortizers();
+    initializeSecurityProcessors();
+  }
+
+  private void initializeSecurityProcessors() {
+    securityProcessors[0] = new NoAuthenticationSecurityProcessor();
+    securityProcessors[1] = new AuthenticationSecurityProcessor();
+  }
+
+  private void initializeAuthenticators() {
+    authenticators[0] = new NoOpAuthenticator();
+    authenticators[1] = new InvalidConfigAuthenticator();
+  }
+
+  private void initializeAuthortizers() {
+    authorizers[0] = new NoOpAuthorizer();
+  }
+
+  public SecurityProcessor lookupProcessor(SecurityService securityService) {
+    return isSecurityEnabled(securityService) ? securityProcessors[1] : securityProcessors[0];
+  }
+
+  public Authenticator lookupAuthenticator(SecurityService securityService) {
+    if (securityService.isIntegratedSecurity()) {
+      // no need to care about thread safety, eventually there will only be one authenticator
+      if (authenticators[2] == null) {
+        authenticators[2] = new ProtobufShiroAuthenticator(securityService);
+      }
+      // Simple authenticator...normal shiro
+      return authenticators[2];
+    }
+    if (isLegacySecurity(securityService)) {
+      // Failing authentication...legacy security
+      return authenticators[1];
+    } else {
+      // Noop authenticator...no security
+      return authenticators[0];
+    }
+  }
+
+  public Authorizer lookupAuthorizer(SecurityService securityService) {
+    if (securityService.isIntegratedSecurity()) {
+      // Simple authenticator...normal shiro
+      if (authorizers[1] == null) {
+        authorizers[1] = new ProtobufShiroAuthorizer(securityService);
+      }
+      // Simple authenticator...normal shiro
+      return authorizers[1];
+    }
+    if (isLegacySecurity(securityService)) {
+      // Failing authentication...legacy security
+      // This should never be called.
+      return null;
+    } else {
+      // Noop authenticator...no security
+      return authorizers[0];
+    }
+  }
+
+  private boolean isLegacySecurity(SecurityService securityService) {
+    return securityService.isPeerSecurityRequired() || securityService.isClientSecurityRequired();
+  }
+
+  private boolean isSecurityEnabled(SecurityService securityService) {
+    return securityService.isIntegratedSecurity() || isLegacySecurity(securityService);
+  }
+}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityProcessor.java
index 40d5e9e..a60cd73 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/SecurityProcessor.java
@@ -14,10 +14,13 @@
  */
 package org.apache.geode.internal.protocol.security;
 
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.OperationContext;
 
 public interface SecurityProcessor<RequestMessage> {
-  boolean validateOperation(RequestMessage request, MessageExecutionContext messageExecutionContext,
+  /**
+   * This method will validate an Operation. In the case of a failure, it will throw an exception.
+   */
+  void validateOperation(RequestMessage request, MessageExecutionContext messageExecutionContext,
       OperationContext operationContext);
 }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthorizationSecurityProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/AuthorizationSecurityProcessor.java
similarity index 70%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthorizationSecurityProcessor.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/AuthorizationSecurityProcessor.java
index 7664e8a..59d0084 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/AuthorizationSecurityProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/AuthorizationSecurityProcessor.java
@@ -12,18 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.protocol.protobuf.security.processors;
+package org.apache.geode.internal.protocol.security.processors;
 
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.OperationContext;
-import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.internal.protocol.security.SecurityProcessor;
 
-public class AuthorizationSecurityProcessor implements SecurityProcessor<ClientProtocol.Request> {
+public class AuthorizationSecurityProcessor implements SecurityProcessor<Object> {
   @Override
-  public boolean validateOperation(ClientProtocol.Request request,
-      MessageExecutionContext messageExecutionContext, OperationContext operationContext) {
-    return messageExecutionContext.getAuthorizer().authorize(
+  public void validateOperation(Object request, MessageExecutionContext messageExecutionContext,
+      OperationContext operationContext) {
+    messageExecutionContext.getAuthorizer().authorize(
         messageExecutionContext.getAuthenticationToken(),
         operationContext.getAccessPermissionRequired());
   }
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/NoAuthenticationSecurityProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/NoAuthenticationSecurityProcessor.java
similarity index 65%
rename from geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/NoAuthenticationSecurityProcessor.java
rename to geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/NoAuthenticationSecurityProcessor.java
index d603b1a..4ed23f5 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/security/processors/NoAuthenticationSecurityProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/security/processors/NoAuthenticationSecurityProcessor.java
@@ -12,18 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.protocol.protobuf.security.processors;
+package org.apache.geode.internal.protocol.security.processors;
 
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.OperationContext;
-import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.internal.protocol.security.SecurityProcessor;
 
-public class NoAuthenticationSecurityProcessor
-    implements SecurityProcessor<ClientProtocol.Request> {
+public class NoAuthenticationSecurityProcessor implements SecurityProcessor<Object> {
+
   @Override
-  public boolean validateOperation(ClientProtocol.Request request,
-      MessageExecutionContext messageExecutionContext, OperationContext operationContext) {
-    return true;
+  public void validateOperation(Object request, MessageExecutionContext messageExecutionContext,
+      OperationContext operationContext) {
+    // A truly "no-op" operation :)
   }
 }
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.client.protocol.ClientProtocolService b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.client.protocol.ClientProtocolService
new file mode 100644
index 0000000..2db0521
--- /dev/null
+++ b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.client.protocol.ClientProtocolService
@@ -0,0 +1 @@
+org.apache.geode.internal.protocol.protobuf.ProtobufProtocolService
\ No newline at end of file
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.tier.sockets.ClientProtocolService b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.tier.sockets.ClientProtocolService
deleted file mode 100644
index 207426a..0000000
--- a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.internal.cache.tier.sockets.ClientProtocolService
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.geode.internal.protocol.ProtobufProtocolService
\ No newline at end of file
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
index 8f16935..1414826 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
@@ -28,8 +28,6 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.UnknownHostException;
 
-import org.apache.geode.distributed.internal.SecurityConfig;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentCaptor;
@@ -38,6 +36,7 @@ import org.mockito.Mockito;
 import org.apache.geode.cache.IncompatibleVersionException;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.client.protocol.ClientProtocolProcessor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.CommunicationMode;
 import org.apache.geode.internal.security.SecurityService;
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/AuthenticationIntegrationTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/AuthenticationIntegrationTest.java
index 124abbb..dabfa4c 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/AuthenticationIntegrationTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/AuthenticationIntegrationTest.java
@@ -14,13 +14,11 @@
  */
 package org.apache.geode.internal.protocol;
 
-import static junit.framework.TestCase.fail;
 import static org.apache.geode.internal.protocol.ProtocolErrorCode.AUTHENTICATION_FAILED;
+import static org.apache.geode.internal.protocol.ProtocolErrorCode.UNSUPPORTED_AUTHENTICATION_MODE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -286,7 +284,7 @@ public class AuthenticationIntegrationTest {
     ClientProtocol.Message errorResponse = protobufProtocolSerializer.deserialize(inputStream);
     assertEquals(ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE,
         errorResponse.getResponse().getResponseAPICase());
-    assertEquals(AUTHENTICATION_FAILED.codeValue,
+    assertEquals(UNSUPPORTED_AUTHENTICATION_MODE.codeValue,
         errorResponse.getResponse().getErrorResponse().getError().getErrorCode());
   }
 
@@ -305,7 +303,7 @@ public class AuthenticationIntegrationTest {
     ClientProtocol.Message errorResponse = protobufProtocolSerializer.deserialize(inputStream);
     assertEquals(ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE,
         errorResponse.getResponse().getResponseAPICase());
-    assertEquals(AUTHENTICATION_FAILED.codeValue,
+    assertEquals(UNSUPPORTED_AUTHENTICATION_MODE.codeValue,
         errorResponse.getResponse().getErrorResponse().getError().getErrorCode());
   }
 
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestExecutionContext.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufTestExecutionContext.java
similarity index 66%
rename from geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestExecutionContext.java
rename to geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufTestExecutionContext.java
index 1781865..fc7a081 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestExecutionContext.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufTestExecutionContext.java
@@ -12,18 +12,22 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.protocol.protobuf;
+package org.apache.geode.internal.protocol;
 
 import org.apache.geode.cache.Cache;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.protocol.protobuf.security.processors.NoAuthenticationSecurityProcessor;
+import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.protocol.security.NoOpAuthenticator;
 import org.apache.geode.internal.protocol.security.NoOpAuthorizer;
+import org.apache.geode.internal.protocol.security.processors.NoAuthenticationSecurityProcessor;
 import org.apache.geode.internal.protocol.statistics.NoOpStatistics;
 
 public class ProtobufTestExecutionContext {
-  public static MessageExecutionContext getNoAuthExecutionContext(Cache cache) {
-    return new MessageExecutionContext(cache, new NoOpAuthenticator(), new NoOpAuthorizer(), new Object(),
-        new NoOpStatistics(), new NoAuthenticationSecurityProcessor());
+  public static MessageExecutionContext getNoAuthCacheExecutionContext(Cache cache) {
+    return new MessageExecutionContext(cache, new NoOpAuthenticator(), new NoOpAuthorizer(),
+        new Object(), new NoOpStatistics(), new NoAuthenticationSecurityProcessor());
+  }
+
+  public static MessageExecutionContext getLocatorExecutionContext(InternalLocator locator) {
+    return new MessageExecutionContext(locator, new NoOpStatistics());
   }
 }
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/HandshakerTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/HandshakerTest.java
deleted file mode 100644
index 10bfeaf..0000000
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/HandshakerTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.protocol.protobuf;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.internal.protocol.Handshaker;
-import org.apache.geode.internal.protocol.security.Authenticator;
-import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class HandshakerTest {
-
-  private Map<String, Class<? extends Authenticator>> authenticatorMap;
-  private Handshaker handshaker;
-
-  private static class AuthenticatorMock implements Authenticator {
-
-
-    @Override
-    public Object authenticate(Object o) throws AuthenticationFailedException {
-      return null;
-    }
-  }
-
-  private static class SimpleMock extends AuthenticatorMock {
-  }
-
-  private static class NoopMock extends AuthenticatorMock {
-  }
-
-  @Before
-  public void setUp() {
-    handshaker = new Handshaker();
-    assertFalse(handshaker.completed());
-  }
-
-  @Test
-  public void version1_0IsSupported() throws Exception {
-    HandshakeAPI.HandshakeRequest handshakeRequest = HandshakeAPI.HandshakeRequest.newBuilder()
-        .setVersion(HandshakeAPI.Semver.newBuilder().setMajor(1).setMinor(1)).build();
-
-    ByteArrayInputStream byteArrayInputStream =
-        ProtobufTestUtilities.messageToByteArrayInputStream(handshakeRequest);
-
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-
-    handshaker.processHandshake(byteArrayInputStream, byteArrayOutputStream);
-    // assertTrue(actualAuthenticator instanceof NoopMock);
-
-    assertTrue(handshaker.completed());
-  }
-
-  @Test
-  public void version2NotSupported() throws Exception {
-    HandshakeAPI.HandshakeRequest handshakeRequest = HandshakeAPI.HandshakeRequest.newBuilder()
-        .setVersion(HandshakeAPI.Semver.newBuilder().setMajor(2).setMinor(0))
-        .build();
-
-    ByteArrayInputStream byteArrayInputStream =
-        ProtobufTestUtilities.messageToByteArrayInputStream(handshakeRequest);
-
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-
-    handshaker.processHandshake(byteArrayInputStream, byteArrayOutputStream);
-  }
-
-  @Test
-  public void bogusAuthenticationMode() throws Exception {
-    HandshakeAPI.HandshakeRequest handshakeRequest = HandshakeAPI.HandshakeRequest.newBuilder()
-        .setVersion(HandshakeAPI.Semver.newBuilder().setMajor(1).setMinor(0))
-        .build();
-
-    ByteArrayInputStream byteArrayInputStream =
-        ProtobufTestUtilities.messageToByteArrayInputStream(handshakeRequest);
-
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-
-    handshaker.processHandshake(byteArrayInputStream, byteArrayOutputStream);
-  }
-
-  @Test
-  public void simpleIsSupported() throws Exception {
-    HandshakeAPI.HandshakeRequest handshakeRequest = HandshakeAPI.HandshakeRequest.newBuilder()
-        .setVersion(HandshakeAPI.Semver.newBuilder().setMajor(1).setMinor(0))
-        .build();
-
-    ByteArrayInputStream byteArrayInputStream =
-        ProtobufTestUtilities.messageToByteArrayInputStream(handshakeRequest);
-
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-
-    handshaker.processHandshake(byteArrayInputStream, byteArrayOutputStream);
-    // assertTrue(actualAuthenticator instanceof SimpleMock);
-
-    assertTrue(handshaker.completed());
-  }
-}
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufProtocolServiceJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolServiceJUnitTest.java
similarity index 92%
rename from geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufProtocolServiceJUnitTest.java
rename to geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolServiceJUnitTest.java
index 91e133d..52b78d0 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/ProtobufProtocolServiceJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufProtocolServiceJUnitTest.java
@@ -12,13 +12,14 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.protocol;
+package org.apache.geode.internal.protocol.protobuf;
 
 import static org.junit.Assert.*;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.geode.internal.protocol.protobuf.ProtobufProtocolService;
 import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics;
 import org.apache.geode.internal.statistics.DummyStatisticsFactory;
 import org.apache.geode.test.junit.categories.UnitTest;
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessorTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessorTest.java
index e4be893..d56de00 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessorTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufStreamProcessorTest.java
@@ -15,11 +15,7 @@
 package org.apache.geode.internal.protocol.protobuf;
 
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
-import org.apache.geode.internal.protocol.protobuf.security.processors.NoAuthenticationSecurityProcessor;
-import org.apache.geode.internal.protocol.security.NoOpAuthenticator;
-import org.apache.geode.internal.protocol.security.NoOpAuthorizer;
-import org.apache.geode.internal.protocol.statistics.NoOpStatistics;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 import org.junit.Test;
@@ -42,8 +38,7 @@ public class ProtobufStreamProcessorTest {
 
     ProtobufStreamProcessor protobufStreamProcessor = new ProtobufStreamProcessor();
     InternalCache mockInternalCache = mock(InternalCache.class);
-    protobufStreamProcessor.receiveMessage(inputStream, outputStream, new MessageExecutionContext(
-        mockInternalCache, new NoOpAuthenticator(), new NoOpAuthorizer(), new Object(),
-        new NoOpStatistics(), new NoAuthenticationSecurityProcessor()));
+    protobufStreamProcessor.receiveMessage(inputStream, outputStream,
+        ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(mockInternalCache));
   }
 }
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestUtilities.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestUtilities.java
deleted file mode 100644
index 9a767fd..0000000
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/ProtobufTestUtilities.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.protocol.protobuf;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import com.google.protobuf.GeneratedMessageV3;
-
-public class ProtobufTestUtilities {
-  public static ByteArrayInputStream messageToByteArrayInputStream(GeneratedMessageV3 message)
-      throws IOException {
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    message.writeDelimitedTo(byteArrayOutputStream);
-    return new ByteArrayInputStream(byteArrayOutputStream.toByteArray());
-  }
-
-
-  public static ClientProtocol.Request createProtobufRequestWithGetRegionNamesRequest(
-      RegionAPI.GetRegionNamesRequest getRegionNamesRequest) {
-    return ClientProtocol.Request.newBuilder().setGetRegionNamesRequest(getRegionNamesRequest)
-        .build();
-  }
-
-  public static void verifyHandshake(InputStream inputStream, OutputStream outputStream,
-      HandshakeAPI.AuthenticationMode authenticationMode) throws IOException {
-    buildHandshakeRequest(authenticationMode).writeDelimitedTo(outputStream);
-
-    HandshakeAPI.HandshakeResponse handshakeResponse =
-        HandshakeAPI.HandshakeResponse.parseDelimitedFrom(inputStream);
-
-    assertTrue(handshakeResponse.getOk());
-    assertFalse(handshakeResponse.hasError());
-  }
-
-  public static HandshakeAPI.HandshakeRequest buildHandshakeRequest(
-      HandshakeAPI.AuthenticationMode authenticationMode) {
-    return HandshakeAPI.HandshakeRequest.newBuilder()
-        .setVersion(HandshakeAPI.Semver.newBuilder().setMajor(1).setMinor(0))
-        .build();
-  }
-}
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
index 40b8a7c..bcd98c3 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
@@ -33,10 +33,9 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.CacheLoaderException;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
-import org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.Success;
@@ -77,9 +76,9 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
 
   @Test
   public void processReturnsExpectedValuesForValidKeys() throws Exception {
-    Result<RegionAPI.GetAllResponse,MessageExecutionContext> result =
+    Result<RegionAPI.GetAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(true, false),
-            ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
 
@@ -98,9 +97,9 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processReturnsNoEntriesForNoKeysRequested() throws UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
-    Result<RegionAPI.GetAllResponse,MessageExecutionContext> result =
+    Result<RegionAPI.GetAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(false, false),
-            ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
 
@@ -116,8 +115,8 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
         NO_VALUE_PRESENT_FOR_THIS_KEY));
     RegionAPI.GetAllRequest getAllRequest =
         ProtobufRequestUtilities.createGetAllRequest(TEST_REGION, testKeys);
-    Result<RegionAPI.GetAllResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        getAllRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetAllResponse> result = operationHandler.process(serializationServiceStub,
+        getAllRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
     RegionAPI.GetAllResponse message = result.getMessage();
@@ -131,9 +130,9 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void multipleKeysWhereOneThrows() throws UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
-    Result<RegionAPI.GetAllResponse,MessageExecutionContext> result =
+    Result<RegionAPI.GetAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(true, true),
-            ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
 
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java
index 418191d..6e26fda 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetAvailableServersOperationHandlerJUnitTest.java
@@ -19,7 +19,8 @@ import org.apache.geode.distributed.internal.LocatorLoadSnapshot;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.ServerLocator;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.internal.protocol.MessageExecutionContext;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.protobuf.ServerAPI;
@@ -28,6 +29,7 @@ import org.apache.geode.internal.protocol.Success;
 import org.apache.geode.internal.protocol.statistics.NoOpStatistics;
 import org.apache.geode.internal.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.test.junit.categories.UnitTest;
+
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -61,7 +63,6 @@ public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandl
     ServerLocator serverLocatorAdviseeMock = mock(ServerLocator.class);
     locatorLoadSnapshot = mock(LocatorLoadSnapshot.class);
 
-
     when(internalLocatorMock.getServerLocatorAdvisee()).thenReturn(serverLocatorAdviseeMock);
     when(serverLocatorAdviseeMock.getLoadSnapshot()).thenReturn(locatorLoadSnapshot);
   }
@@ -75,9 +76,7 @@ public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandl
 
     ServerAPI.GetAvailableServersRequest getAvailableServersRequest =
         ProtobufRequestUtilities.createGetAvailableServersRequest();
-    Result operationHandlerResult =
-        operationHandler.process(serializationServiceStub, getAvailableServersRequest,
-            new MessageExecutionContext(internalLocatorMock, new NoOpStatistics()));
+    Result operationHandlerResult = getOperationHandlerResult(getAvailableServersRequest);
     assertTrue(operationHandlerResult instanceof Success);
     ValidateGetAvailableServersResponse(
         (GetAvailableServersResponse) operationHandlerResult.getMessage());
@@ -90,15 +89,20 @@ public class GetAvailableServersOperationHandlerJUnitTest extends OperationHandl
 
     ServerAPI.GetAvailableServersRequest getAvailableServersRequest =
         ProtobufRequestUtilities.createGetAvailableServersRequest();
-    Result operationHandlerResult =
-        operationHandler.process(serializationServiceStub, getAvailableServersRequest,
-            new MessageExecutionContext(internalLocatorMock, new NoOpStatistics()));
+    Result operationHandlerResult = getOperationHandlerResult(getAvailableServersRequest);
     assertTrue(operationHandlerResult instanceof Success);
     GetAvailableServersResponse availableServersResponse =
         (GetAvailableServersResponse) operationHandlerResult.getMessage();
     assertEquals(0, availableServersResponse.getServersCount());
   }
 
+  private Result getOperationHandlerResult(
+      ServerAPI.GetAvailableServersRequest getAvailableServersRequest)
+      throws InvalidExecutionContextException {
+    return operationHandler.process(serializationServiceStub, getAvailableServersRequest,
+        ProtobufTestExecutionContext.getLocatorExecutionContext(internalLocatorMock));
+  }
+
   private void ValidateGetAvailableServersResponse(
       GetAvailableServersResponse getAvailableServersResponse) {
     assertEquals(2, getAvailableServersResponse.getServersCount());
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
index ba42f63..5e09ec6 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
@@ -14,10 +14,12 @@
  */
 package org.apache.geode.internal.protocol.protobuf.operations;
 
-import static org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext.getNoAuthExecutionContext;
+import static org.apache.geode.internal.Assert.assertTrue;
+import static org.apache.geode.internal.protocol.ProtobufTestExecutionContext.getNoAuthCacheExecutionContext;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
@@ -29,7 +31,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
@@ -64,9 +65,9 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
   @Test
   public void processReturnsCacheRegions() throws UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
-    Result<RegionAPI.GetRegionNamesResponse,MessageExecutionContext> result = operationHandler.process(
+    Result<RegionAPI.GetRegionNamesResponse> result = operationHandler.process(
         serializationServiceStub, ProtobufRequestUtilities.createGetRegionNamesRequest(),
-        getNoAuthExecutionContext(cacheStub));
+        getNoAuthCacheExecutionContext(cacheStub));
     Assert.assertTrue(result instanceof Success);
 
     RegionAPI.GetRegionNamesResponse getRegionsResponse = result.getMessage();
@@ -77,10 +78,15 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
     String name2 = getRegionsResponse.getRegions(1);
     String name3 = getRegionsResponse.getRegions(2);
     Assert.assertTrue("The same region was returned multiple times",
-        name1 != name2 && name1 != name3 && name2 != name3);
-    Assert.assertTrue(name1 == TEST_REGION1 || name1 == TEST_REGION2 || name1 == TEST_REGION3);
-    Assert.assertTrue(name2 == TEST_REGION1 || name2 == TEST_REGION2 || name2 == TEST_REGION3);
-    Assert.assertTrue(name3 == TEST_REGION1 || name3 == TEST_REGION2 || name3 == TEST_REGION3);
+        !name1.equals(name2) && !name1.equals(name3) && !name2.equals(name3));
+    ArrayList arrayList = new ArrayList();
+    arrayList.add(TEST_REGION1);
+    arrayList.add(TEST_REGION2);
+    arrayList.add(TEST_REGION3);
+
+    assertTrue(arrayList.contains(name1));
+    assertTrue(arrayList.contains(name2));
+    assertTrue(arrayList.contains(name3));
   }
 
   @Test
@@ -88,9 +94,9 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
     Cache emptyCache = mock(Cache.class);;
     when(emptyCache.rootRegions())
         .thenReturn(Collections.unmodifiableSet(new HashSet<Region<String, String>>()));
-    Result<RegionAPI.GetRegionNamesResponse,MessageExecutionContext> result = operationHandler.process(
+    Result<RegionAPI.GetRegionNamesResponse> result = operationHandler.process(
         serializationServiceStub, ProtobufRequestUtilities.createGetRegionNamesRequest(),
-        getNoAuthExecutionContext(emptyCache));
+        getNoAuthCacheExecutionContext(emptyCache));
     Assert.assertTrue(result instanceof Success);
 
     RegionAPI.GetRegionNamesResponse getRegionsResponse = result.getMessage();
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
index f2aebb5..6775f06 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.protocol.protobuf.operations;
 
-import static org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext.getNoAuthExecutionContext;
+import static org.apache.geode.internal.protocol.ProtobufTestExecutionContext.getNoAuthCacheExecutionContext;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -31,7 +31,6 @@ import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.protocol.MessageUtil;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
@@ -70,8 +69,8 @@ public class GetRegionRequestOperationHandlerJUnitTest extends OperationHandlerJ
     when(regionAttributesStub.getScope()).thenReturn(Scope.DISTRIBUTED_ACK);
 
 
-    Result<RegionAPI.GetRegionResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        MessageUtil.makeGetRegionRequest(TEST_REGION1), getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetRegionResponse> result = operationHandler.process(serializationServiceStub,
+        MessageUtil.makeGetRegionRequest(TEST_REGION1), getNoAuthCacheExecutionContext(cacheStub));
     RegionAPI.GetRegionResponse response = result.getMessage();
     BasicTypes.Region region = response.getRegion();
     Assert.assertEquals(TEST_REGION1, region.getName());
@@ -93,8 +92,9 @@ public class GetRegionRequestOperationHandlerJUnitTest extends OperationHandlerJ
     when(emptyCache.rootRegions())
         .thenReturn(Collections.unmodifiableSet(new HashSet<Region<String, String>>()));
     String unknownRegionName = "UNKNOWN_REGION";
-    Result<RegionAPI.GetRegionResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        MessageUtil.makeGetRegionRequest(unknownRegionName), getNoAuthExecutionContext(emptyCache));
+    Result<RegionAPI.GetRegionResponse> result = operationHandler.process(serializationServiceStub,
+        MessageUtil.makeGetRegionRequest(unknownRegionName),
+        getNoAuthCacheExecutionContext(emptyCache));
     Assert.assertTrue(result instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
         result.getErrorMessage().getError().getErrorCode());
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
index f3e80bf..302232c 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
@@ -25,10 +25,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.Failure;
-import org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.internal.protocol.ProtocolErrorCode;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
@@ -67,8 +66,8 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void processReturnsTheEncodedValueFromTheRegion() throws Exception {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, false, false);
-    Result<RegionAPI.GetResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        getRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetResponse> result = operationHandler.process(serializationServiceStub,
+        getRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(result instanceof Success);
     Assert.assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT,
@@ -80,8 +79,8 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void processReturnsUnsucessfulResponseForInvalidRegion() throws Exception {
     RegionAPI.GetRequest getRequest = generateTestRequest(true, false, false);
-    Result<RegionAPI.GetResponse,MessageExecutionContext> response = operationHandler.process(serializationServiceStub,
-        getRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
+        getRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(response instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -91,8 +90,8 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void processReturnsKeyNotFoundWhenKeyIsNotFound() throws Exception {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, true, false);
-    Result<RegionAPI.GetResponse,MessageExecutionContext> response = operationHandler.process(serializationServiceStub,
-        getRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
+        getRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(response instanceof Success);
   }
@@ -100,8 +99,8 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void processReturnsLookupFailureWhenKeyFoundWithNoValue() throws Exception {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, false, true);
-    Result<RegionAPI.GetResponse,MessageExecutionContext> response = operationHandler.process(serializationServiceStub,
-        getRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
+        getRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(response instanceof Success);
   }
@@ -119,8 +118,8 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
         .setCustomEncodedValue(customEncodedValueBuilder).build();
     RegionAPI.GetRequest getRequest =
         ProtobufRequestUtilities.createGetRequest(TEST_REGION, encodedKey).getGetRequest();
-    Result<RegionAPI.GetResponse,MessageExecutionContext> response = operationHandler.process(serializationServiceStub,
-        getRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
+        getRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(response instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
index 9e3e406..80b15ff 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.protocol.protobuf.operations;
 
-import static org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext.getNoAuthExecutionContext;
+import static org.apache.geode.internal.protocol.ProtobufTestExecutionContext.getNoAuthCacheExecutionContext;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.any;
@@ -31,7 +31,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
@@ -71,8 +70,8 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   public void processInsertsMultipleValidEntriesInCache() throws Exception {
     PutAllRequestOperationHandler operationHandler = new PutAllRequestOperationHandler();
 
-    Result<RegionAPI.PutAllResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(false, true), getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutAllResponse> result = operationHandler.process(serializationServiceStub,
+        generateTestRequest(false, true), getNoAuthCacheExecutionContext(cacheStub));
 
     Assert.assertTrue(result instanceof Success);
 
@@ -85,8 +84,8 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   public void processWithInvalidEntrySucceedsAndReturnsFailedKey() throws Exception {
     PutAllRequestOperationHandler operationHandler = new PutAllRequestOperationHandler();
 
-    Result<RegionAPI.PutAllResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(true, true), getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutAllResponse> result = operationHandler.process(serializationServiceStub,
+        generateTestRequest(true, true), getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
     verify(regionMock).put(TEST_KEY1, TEST_VALUE1);
@@ -104,8 +103,8 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   public void processWithNoEntriesPasses() throws Exception {
     PutAllRequestOperationHandler operationHandler = new PutAllRequestOperationHandler();
 
-    Result<RegionAPI.PutAllResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(false, false), getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutAllResponse> result = operationHandler.process(serializationServiceStub,
+        generateTestRequest(false, false), getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
 
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
index 4dec3f0..04ce826 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
@@ -29,10 +29,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.Failure;
-import org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.internal.protocol.ProtocolErrorCode;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
@@ -63,8 +62,9 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   @Test
   public void test_puttingTheEncodedEntryIntoRegion() throws Exception {
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
-    Result<RegionAPI.PutResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutResponse> result =
+        operationHandler.process(serializationServiceStub, generateTestRequest(),
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
 
@@ -92,8 +92,8 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
     BasicTypes.Entry testEntry = ProtobufUtilities.createEntry(encodedKey, testValue);
     RegionAPI.PutRequest putRequest =
         ProtobufRequestUtilities.createPutRequest(TEST_REGION, testEntry).getPutRequest();
-    Result<RegionAPI.PutResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        putRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutResponse> result = operationHandler.process(serializationServiceStub,
+        putRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,
@@ -104,8 +104,9 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
   public void test_RegionNotFound() throws Exception {
     when(cacheStub.getRegion(TEST_REGION)).thenReturn(null);
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
-    Result<RegionAPI.PutResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutResponse> result =
+        operationHandler.process(serializationServiceStub, generateTestRequest(),
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -117,8 +118,9 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
     when(regionMock.put(any(), any())).thenThrow(ClassCastException.class);
 
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
-    Result<RegionAPI.PutResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.PutResponse> result =
+        operationHandler.process(serializationServiceStub, generateTestRequest(),
+            ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.CONSTRAINT_VIOLATION.codeValue,
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
index e5166d0..3b8fc66 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
@@ -27,11 +27,10 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
-import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.BasicTypes;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 import org.apache.geode.internal.protocol.Failure;
-import org.apache.geode.internal.protocol.protobuf.ProtobufTestExecutionContext;
+import org.apache.geode.internal.protocol.ProtobufTestExecutionContext;
 import org.apache.geode.internal.protocol.ProtocolErrorCode;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.internal.protocol.Result;
@@ -68,8 +67,8 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processValidKeyRemovesTheEntryAndReturnSuccess() throws Exception {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(false, false).getRemoveRequest();
-    Result<RegionAPI.RemoveResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        removeRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
+        removeRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
     verify(regionStub).remove(TEST_KEY);
@@ -78,8 +77,8 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processReturnsUnsucessfulResponseForInvalidRegion() throws Exception {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(true, false).getRemoveRequest();
-    Result<RegionAPI.RemoveResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        removeRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
+        removeRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -89,8 +88,8 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processReturnsSuccessWhenKeyIsNotFound() throws Exception {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(false, true).getRemoveRequest();
-    Result<RegionAPI.RemoveResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        removeRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
+        removeRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Success);
   }
@@ -109,8 +108,8 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
 
     RegionAPI.RemoveRequest removeRequest =
         ProtobufRequestUtilities.createRemoveRequest(TEST_REGION, encodedKey).getRemoveRequest();;
-    Result<RegionAPI.RemoveResponse,MessageExecutionContext> result = operationHandler.process(serializationServiceStub,
-        removeRequest, ProtobufTestExecutionContext.getNoAuthExecutionContext(cacheStub));
+    Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
+        removeRequest, ProtobufTestExecutionContext.getNoAuthCacheExecutionContext(cacheStub));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <co...@geode.apache.org>.