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

[46/48] geode git commit: GEODE-3447 Implement client authorization for the new protocol

GEODE-3447 Implement client authorization for the new protocol

Implementation of authorization checks for the new protocol.


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

Branch: refs/heads/feature/GEODE-3447
Commit: a2b678aa06af73d218da6e9ae8d95f2942d23a7b
Parents: c0f6c84
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Aug 17 15:22:57 2017 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Aug 22 14:46:13 2017 -0700

----------------------------------------------------------------------
 .../sockets/ClientProtocolMessageHandler.java   |   3 +
 .../geode/security/NoOpStreamAuthenticator.java |   7 +-
 .../geode/security/NoOpStreamAuthorizer.java    |  26 +++
 .../geode/security/StreamAuthenticator.java     |  10 +-
 .../apache/geode/security/StreamAuthorizer.java |   8 +
 .../GenericProtocolServerConnectionTest.java    |   2 +-
 .../protocol/protobuf/OperationContext.java     |  10 +-
 .../protocol/protobuf/ProtobufOpsProcessor.java |  10 +
 .../protobuf/ProtobufSimpleAuthenticator.java   |  18 +-
 .../protobuf/ProtobufSimpleAuthorizer.java      |  42 ++++
 .../protobuf/ProtobufStreamProcessor.java       |   1 +
 .../registry/OperationContextRegistry.java      |  39 +++-
 .../protocol/AuthorizationIntegrationTest.java  | 205 +++++++++++++++++++
 .../protobuf/ProtobufStreamProcessorTest.java   |   1 +
 14 files changed, 361 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
index 0ced3aa..0d1dfd9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProtocolMessageHandler.java
@@ -15,6 +15,9 @@
 
 package org.apache.geode.internal.cache.tier.sockets;
 
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.security.StreamAuthorizer;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java b/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java
index 0a6dde1..62f49fe 100644
--- a/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java
@@ -23,8 +23,6 @@ import java.io.OutputStream;
  * returns true.
  */
 public class NoOpStreamAuthenticator implements StreamAuthenticator {
-
-
   @Override
   public void receiveMessage(InputStream inputStream, OutputStream outputStream,
       SecurityManager securityManager) throws IOException {
@@ -37,6 +35,11 @@ public class NoOpStreamAuthenticator implements StreamAuthenticator {
   }
 
   @Override
+  public StreamAuthorizer getAuthorizer() {
+    return new NoOpStreamAuthorizer();
+  }
+
+  @Override
   public String implementationID() {
     return "NOOP";
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthorizer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthorizer.java b/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthorizer.java
new file mode 100644
index 0000000..1b21576
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthorizer.java
@@ -0,0 +1,26 @@
+/*
+ * 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.security;
+
+/**
+ * An implementation of {@link StreamAuthorizer} that doesn't use its parameters and always returns
+ * true.
+ */
+public class NoOpStreamAuthorizer implements StreamAuthorizer {
+  @Override
+  public boolean authorize(ResourcePermission permissionRequested) {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java b/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java
index 7db1a2b..5cbc4f8 100644
--- a/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.security;
 
-import org.apache.geode.security.SecurityManager;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -46,6 +44,14 @@ public interface StreamAuthenticator {
   boolean isAuthenticated();
 
   /**
+   * Return an authorization object which can be used to determine which permissions this stream has
+   * according to the provided securityManager.
+   *
+   * Calling this before authentication has succeeded may result in a null return object.
+   */
+  StreamAuthorizer getAuthorizer();
+
+  /**
    * @return a unique identifier for this particular implementation (NOOP, PASSTHROUGH, etc.)
    */
   String implementationID();

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/main/java/org/apache/geode/security/StreamAuthorizer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/StreamAuthorizer.java b/geode-core/src/main/java/org/apache/geode/security/StreamAuthorizer.java
new file mode 100644
index 0000000..c6cef33
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/StreamAuthorizer.java
@@ -0,0 +1,8 @@
+package org.apache.geode.security;
+
+/**
+ * Created by bschuchardt on 8/17/17.
+ */
+public interface StreamAuthorizer {
+  boolean authorize(ResourcePermission permissionRequested);
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
index 383fbf0..6817b13 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
@@ -53,7 +53,7 @@ public class GenericProtocolServerConnectionTest {
     when(socketMock.getInetAddress()).thenReturn(InetAddress.getByName("localhost"));
 
     ClientProtocolMessageHandler clientProtocolMock = mock(ClientProtocolMessageHandler.class);
-    doThrow(new IOException()).when(clientProtocolMock).receiveMessage(any(), any(), any());
+    doThrow(new IOException()).when(clientProtocolMock).receiveMessage(any(), any(), any(), any());
 
     return new GenericProtocolServerConnection(socketMock, mock(InternalCache.class),
         mock(CachedRegionHelper.class), mock(CacheServerStats.class), 0, 0, "",

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/OperationContext.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/OperationContext.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/OperationContext.java
index 5191007..488dfc4 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/OperationContext.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/OperationContext.java
@@ -19,6 +19,7 @@ import java.util.function.Function;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.protocol.operations.OperationHandler;
+import org.apache.geode.security.ResourcePermission;
 
 @Experimental
 public class OperationContext<OperationRequest, OperationResponse> {
@@ -26,14 +27,17 @@ public class OperationContext<OperationRequest, OperationResponse> {
   private final Function<ClientProtocol.Request, OperationRequest> fromRequest;
   private final Function<OperationResponse, ClientProtocol.Response.Builder> toResponse;
   private final Function<BasicTypes.ErrorResponse, ClientProtocol.Response.Builder> toErrorResponse;
+  private final ResourcePermission accessPermissionRequired;
 
   public OperationContext(Function<ClientProtocol.Request, OperationRequest> fromRequest,
       OperationHandler<OperationRequest, OperationResponse> operationHandler,
-      Function<OperationResponse, ClientProtocol.Response.Builder> toResponse) {
+      Function<OperationResponse, ClientProtocol.Response.Builder> toResponse,
+      ResourcePermission permissionRequired) {
     this.operationHandler = operationHandler;
     this.fromRequest = fromRequest;
     this.toResponse = toResponse;
     this.toErrorResponse = OperationContext::makeErrorBuilder;
+    accessPermissionRequired = permissionRequired;
   }
 
   public static ClientProtocol.Response.Builder makeErrorBuilder(
@@ -56,4 +60,8 @@ public class OperationContext<OperationRequest, OperationResponse> {
   public Function<BasicTypes.ErrorResponse, ClientProtocol.Response.Builder> getToErrorResponse() {
     return toErrorResponse;
   }
+
+  public ResourcePermission getAccessPermissionRequired() {
+    return accessPermissionRequired;
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufOpsProcessor.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufOpsProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufOpsProcessor.java
index 3619e0d..43601d7 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufOpsProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufOpsProcessor.java
@@ -19,6 +19,7 @@ import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.cache.tier.sockets.InvalidExecutionContextException;
 import org.apache.geode.protocol.protobuf.registry.OperationContextRegistry;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufResponseUtilities;
+import org.apache.geode.security.StreamAuthorizer;
 import org.apache.geode.serialization.SerializationService;
 
 /**
@@ -51,6 +52,15 @@ public class ProtobufOpsProcessor {
           ProtocolErrorCode.UNSUPPORTED_OPERATION.codeValue,
           "Invalid execution context found for operation."));
     }
+    Result result;
+    if (authorizer.authorize(operationContext.getAccessPermissionRequired())) {
+      result = operationContext.getOperationHandler().process(serializationService,
+          operationContext.getFromRequest().apply(request), cache);
+    } else {
+      result = Failure.of(ProtobufResponseUtilities.makeErrorResponse(
+          ProtocolErrorCode.AUTHORIZATION_FAILED.codeValue,
+          "User isn't authorized for this operation."));
+    }
 
     builder = (ClientProtocol.Response.Builder) result.map(operationContext.getToResponse(),
         operationContext.getToErrorResponse());

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticator.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticator.java
index 00285bb..4b702ea 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticator.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticator.java
@@ -18,6 +18,7 @@ import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.security.StreamAuthenticator;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.SecurityManager;
+import org.apache.geode.security.StreamAuthorizer;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -27,6 +28,7 @@ import java.util.Properties;
 
 public class ProtobufSimpleAuthenticator implements StreamAuthenticator {
   private boolean authenticated;
+  private ProtobufSimpleAuthorizer authorizer = null;
 
   @Override
   public void receiveMessage(InputStream inputStream, OutputStream outputStream,
@@ -41,20 +43,28 @@ public class ProtobufSimpleAuthenticator implements StreamAuthenticator {
     properties.setProperty(ResourceConstants.USER_NAME, authenticationRequest.getUsername());
     properties.setProperty(ResourceConstants.PASSWORD, authenticationRequest.getPassword());
 
+    authorizer = null; // authenticating a new user clears current authorizer
     try {
       Object principal = securityManager.authenticate(properties);
-      authenticated = principal != null;
+      if (principal != null) {
+        authorizer = new ProtobufSimpleAuthorizer(principal, securityManager);
+      }
     } catch (AuthenticationFailedException e) {
-      authenticated = false;
+      authorizer = null;
     }
 
-    AuthenticationAPI.SimpleAuthenticationResponse.newBuilder().setAuthenticated(authenticated)
+    AuthenticationAPI.SimpleAuthenticationResponse.newBuilder().setAuthenticated(isAuthenticated())
         .build().writeDelimitedTo(outputStream);
   }
 
   @Override
   public boolean isAuthenticated() {
-    return authenticated;
+    return authorizer != null;
+  }
+
+  @Override
+  public StreamAuthorizer getAuthorizer() {
+    return authorizer;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthorizer.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthorizer.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthorizer.java
new file mode 100644
index 0000000..872632a
--- /dev/null
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthorizer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.protocol.protobuf;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Properties;
+
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.security.SecurityManager;
+import org.apache.geode.security.StreamAuthenticator;
+import org.apache.geode.security.StreamAuthorizer;
+
+public class ProtobufSimpleAuthorizer implements StreamAuthorizer {
+  private final Object authenticatedPrincipal;
+  private final SecurityManager securityManager;
+
+  public ProtobufSimpleAuthorizer(Object authenticatedPrincipal, SecurityManager securityManager) {
+    this.authenticatedPrincipal = authenticatedPrincipal;
+    this.securityManager = securityManager;
+  }
+
+  @Override
+  public boolean authorize(ResourcePermission permissionRequested) {
+    return securityManager.authorize(authenticatedPrincipal, permissionRequested);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessor.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessor.java
index accb899..8f39cd5 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessor.java
@@ -26,6 +26,7 @@ import org.apache.geode.protocol.exception.InvalidProtocolMessageException;
 import org.apache.geode.protocol.protobuf.registry.OperationContextRegistry;
 import org.apache.geode.protocol.protobuf.serializer.ProtobufProtocolSerializer;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
+import org.apache.geode.security.StreamAuthorizer;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 
 /**

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/registry/OperationContextRegistry.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/registry/OperationContextRegistry.java b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/registry/OperationContextRegistry.java
index dfe975c..6b541eb 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/registry/OperationContextRegistry.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/protocol/protobuf/registry/OperationContextRegistry.java
@@ -30,6 +30,7 @@ import org.apache.geode.protocol.protobuf.operations.GetRequestOperationHandler;
 import org.apache.geode.protocol.protobuf.operations.PutAllRequestOperationHandler;
 import org.apache.geode.protocol.protobuf.operations.PutRequestOperationHandler;
 import org.apache.geode.protocol.protobuf.operations.RemoveRequestOperationHandler;
+import org.apache.geode.security.ResourcePermission;
 
 @Experimental
 public class OperationContextRegistry {
@@ -47,41 +48,57 @@ public class OperationContextRegistry {
     operationContexts.put(RequestAPICase.GETREQUEST,
         new OperationContext<>(ClientProtocol.Request::getGetRequest,
             new GetRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setGetResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setGetResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.READ)));
 
     operationContexts.put(RequestAPICase.GETALLREQUEST,
         new OperationContext<>(ClientProtocol.Request::getGetAllRequest,
             new GetAllRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setGetAllResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setGetAllResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.READ)));
 
     operationContexts.put(RequestAPICase.PUTREQUEST,
         new OperationContext<>(ClientProtocol.Request::getPutRequest,
             new PutRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setPutResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setPutResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.WRITE)));
 
     operationContexts.put(RequestAPICase.PUTALLREQUEST,
         new OperationContext<>(ClientProtocol.Request::getPutAllRequest,
             new PutAllRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setPutAllResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setPutAllResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.WRITE)));
 
     operationContexts.put(RequestAPICase.REMOVEREQUEST,
         new OperationContext<>(ClientProtocol.Request::getRemoveRequest,
             new RemoveRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setRemoveResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setRemoveResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.WRITE)));
 
     operationContexts.put(RequestAPICase.GETREGIONNAMESREQUEST,
         new OperationContext<>(ClientProtocol.Request::getGetRegionNamesRequest,
             new GetRegionNamesRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setGetRegionNamesResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setGetRegionNamesResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.READ)));
 
     operationContexts.put(RequestAPICase.GETREGIONREQUEST,
         new OperationContext<>(ClientProtocol.Request::getGetRegionRequest,
             new GetRegionRequestOperationHandler(),
-            opsResp -> ClientProtocol.Response.newBuilder().setGetRegionResponse(opsResp)));
+            opsResp -> ClientProtocol.Response.newBuilder().setGetRegionResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.DATA,
+                ResourcePermission.Operation.READ)));
 
-    operationContexts.put(RequestAPICase.GETAVAILABLESERVERSREQUEST, new OperationContext<>(
-        ClientProtocol.Request::getGetAvailableServersRequest,
-        new GetAvailableServersOperationHandler(),
-        opsResp -> ClientProtocol.Response.newBuilder().setGetAvailableServersResponse(opsResp)));
+    operationContexts.put(RequestAPICase.GETAVAILABLESERVERSREQUEST,
+        new OperationContext<>(ClientProtocol.Request::getGetAvailableServersRequest,
+            new GetAvailableServersOperationHandler(),
+            opsResp -> ClientProtocol.Response.newBuilder().setGetAvailableServersResponse(opsResp),
+            new ResourcePermission(ResourcePermission.Resource.CLUSTER,
+                ResourcePermission.Operation.READ)));
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthorizationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthorizationIntegrationTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthorizationIntegrationTest.java
new file mode 100644
index 0000000..9df23d6
--- /dev/null
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/AuthorizationIntegrationTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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.protocol;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.same;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.awaitility.Awaitility;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.protocol.protobuf.AuthenticationAPI;
+import org.apache.geode.protocol.protobuf.ClientProtocol;
+import org.apache.geode.protocol.protobuf.ProtobufSerializationService;
+import org.apache.geode.protocol.protobuf.ProtocolErrorCode;
+import org.apache.geode.protocol.protobuf.RegionAPI;
+import org.apache.geode.protocol.protobuf.serializer.ProtobufProtocolSerializer;
+import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.security.SecurityManager;
+import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class AuthorizationIntegrationTest {
+
+  private static final String TEST_USERNAME = "bob";
+  private static final String TEST_PASSWORD = "bobspassword";
+  public static final String TEST_REGION = "testRegion";
+
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private Cache cache;
+  private int cacheServerPort;
+  private CacheServer cacheServer;
+  private Socket socket;
+  private OutputStream outputStream;
+  private ProtobufSerializationService serializationService;
+  private InputStream inputStream;
+  private ProtobufProtocolSerializer protobufProtocolSerializer;
+  private Object securityPrincipal;
+  private SecurityManager mockSecurityManager;
+  private String testRegion;
+  public static final ResourcePermission READ_PERMISSION =
+      new ResourcePermission(ResourcePermission.Resource.DATA, ResourcePermission.Operation.READ);
+  public static final ResourcePermission WRITE_PERMISSION =
+      new ResourcePermission(ResourcePermission.Resource.DATA, ResourcePermission.Operation.WRITE);
+
+  @Before
+  public void setUp() throws IOException, CodecAlreadyRegisteredForTypeException {
+    Properties expectedAuthProperties = new Properties();
+    expectedAuthProperties.setProperty("username", TEST_USERNAME);
+    expectedAuthProperties.setProperty("password", TEST_PASSWORD);
+
+    securityPrincipal = new Object();
+    mockSecurityManager = mock(SecurityManager.class);
+    when(mockSecurityManager.authenticate(expectedAuthProperties)).thenReturn(securityPrincipal);
+
+    Properties properties = new Properties();
+    CacheFactory cacheFactory = new CacheFactory(properties);
+    cacheFactory.set("mcast-port", "0"); // sometimes it isn't due to other tests.
+
+    cacheFactory.setSecurityManager(mockSecurityManager);
+    cache = cacheFactory.create();
+
+    cacheServer = cache.addCacheServer();
+    cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    cacheServer.setPort(cacheServerPort);
+    cacheServer.start();
+
+    cache.createRegionFactory().create(TEST_REGION);
+
+    System.setProperty("geode.feature-protobuf-protocol", "true");
+    System.setProperty("geode.protocol-authentication-mode", "SIMPLE");
+    socket = new Socket("localhost", cacheServerPort);
+
+    Awaitility.await().atMost(5, TimeUnit.SECONDS).until(socket::isConnected);
+    outputStream = socket.getOutputStream();
+    inputStream = socket.getInputStream();
+    outputStream.write(110);
+
+    serializationService = new ProtobufSerializationService();
+    protobufProtocolSerializer = new ProtobufProtocolSerializer();
+
+    when(mockSecurityManager.authorize(same(securityPrincipal), any())).thenReturn(false);
+    AuthenticationAPI.SimpleAuthenticationRequest authenticationRequest =
+        AuthenticationAPI.SimpleAuthenticationRequest.newBuilder().setUsername(TEST_USERNAME)
+            .setPassword(TEST_PASSWORD).build();
+    authenticationRequest.writeDelimitedTo(outputStream);
+
+    AuthenticationAPI.SimpleAuthenticationResponse authenticationResponse =
+        AuthenticationAPI.SimpleAuthenticationResponse.parseDelimitedFrom(inputStream);
+    assertTrue(authenticationResponse.getAuthenticated());
+  }
+
+  @After
+  public void shutDown() throws IOException {
+    cache.close();
+    socket.close();
+  }
+
+
+  @Test
+  public void validateNoPermissions() throws Exception {
+    when(mockSecurityManager.authorize(securityPrincipal, READ_PERMISSION)).thenReturn(false);
+    when(mockSecurityManager.authorize(securityPrincipal, WRITE_PERMISSION)).thenReturn(false);
+
+    verifyOperations(false, false);
+  }
+
+  @Test
+  public void validateWritePermission() throws Exception {
+    when(mockSecurityManager.authorize(securityPrincipal, READ_PERMISSION)).thenReturn(false);
+    when(mockSecurityManager.authorize(securityPrincipal, WRITE_PERMISSION)).thenReturn(true);
+
+    verifyOperations(false, true);
+  }
+
+  @Test
+  public void validateReadPermission() throws Exception {
+    when(mockSecurityManager.authorize(securityPrincipal, READ_PERMISSION)).thenReturn(true);
+    when(mockSecurityManager.authorize(securityPrincipal, WRITE_PERMISSION)).thenReturn(false);
+
+    verifyOperations(true, false);
+  }
+
+  @Test
+  public void validateReadAndWritePermission() throws Exception {
+    when(mockSecurityManager.authorize(securityPrincipal, READ_PERMISSION)).thenReturn(true);
+    when(mockSecurityManager.authorize(securityPrincipal, WRITE_PERMISSION)).thenReturn(true);
+
+    verifyOperations(true, true);
+  }
+
+  private void verifyOperations(boolean readAllowed, boolean writeAllowed) throws Exception {
+    ClientProtocol.Message getRegionsMessage =
+        ClientProtocol.Message.newBuilder().setRequest(ClientProtocol.Request.newBuilder()
+            .setGetRegionNamesRequest(RegionAPI.GetRegionNamesRequest.newBuilder())).build();
+    validateOperationAuthorized(getRegionsMessage, inputStream, outputStream,
+        readAllowed ? ClientProtocol.Response.ResponseAPICase.GETREGIONNAMESRESPONSE
+            : ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE);
+
+    ClientProtocol.Message putMessage = ClientProtocol.Message.newBuilder()
+        .setRequest(ClientProtocol.Request.newBuilder()
+            .setPutRequest(RegionAPI.PutRequest.newBuilder().setRegionName(TEST_REGION).setEntry(
+                ProtobufUtilities.createEntry(serializationService, "TEST_KEY", "TEST_VALUE"))))
+        .build();
+    validateOperationAuthorized(putMessage, inputStream, outputStream,
+        writeAllowed ? ClientProtocol.Response.ResponseAPICase.PUTRESPONSE
+            : ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE);
+
+    ClientProtocol.Message removeMessage = ClientProtocol.Message.newBuilder()
+        .setRequest(ClientProtocol.Request.newBuilder()
+            .setRemoveRequest(RegionAPI.RemoveRequest.newBuilder().setRegionName(TEST_REGION)
+                .setKey(ProtobufUtilities.createEncodedValue(serializationService, "TEST_KEY"))))
+        .build();
+    validateOperationAuthorized(removeMessage, inputStream, outputStream,
+        writeAllowed ? ClientProtocol.Response.ResponseAPICase.REMOVERESPONSE
+            : ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE);
+  }
+
+  private void validateOperationAuthorized(ClientProtocol.Message message, InputStream inputStream,
+      OutputStream outputStream, ClientProtocol.Response.ResponseAPICase expectedResponseType)
+      throws Exception {
+    protobufProtocolSerializer.serialize(message, outputStream);
+    ClientProtocol.Message response = protobufProtocolSerializer.deserialize(inputStream);
+    assertEquals(expectedResponseType, response.getResponse().getResponseAPICase());
+    if (expectedResponseType == ClientProtocol.Response.ResponseAPICase.ERRORRESPONSE) {
+      assertEquals(ProtocolErrorCode.AUTHORIZATION_FAILED.codeValue,
+          response.getResponse().getErrorResponse().getError().getErrorCode());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/a2b678aa/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessorTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessorTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessorTest.java
index 16eb48b..668d280 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessorTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufStreamProcessorTest.java
@@ -27,6 +27,7 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
+import org.apache.geode.security.StreamAuthorizer;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)