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/28 21:46:19 UTC

geode git commit: GEODE-3529 move new client/server security classes to a different package

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-3529 [created] ed3ac98ce


GEODE-3529 move new client/server security classes to a different package

The new classes/interfaces have been moved to org.apache.geode.security.server.

A package.html is now provided that describes the purpose of the package.

The word "Stream" has been removed from the names of these classes.  Those
names make sense in the context of the Protobuf support classes in
org.apache.geode.internal.cache.tier.sockets classes but not when taken
as a separate group of classes.

The method Authenticator.receiveMessage() has been renamed to
Authenticator.authenticate().


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

Branch: refs/heads/feature/GEODE-3529
Commit: ed3ac98cec1610bde2c1cbcad5a0ff542296ea2a
Parents: 9e5fad2
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Aug 28 14:43:45 2017 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Aug 28 14:43:45 2017 -0700

----------------------------------------------------------------------
 .../sockets/ClientProtocolMessageHandler.java   |  3 -
 .../GenericProtocolServerConnection.java        |  8 +--
 .../tier/sockets/MessageExecutionContext.java   | 14 ++---
 .../tier/sockets/ServerConnectionFactory.java   | 15 +++--
 .../geode/security/NoOpStreamAuthenticator.java | 46 ---------------
 .../geode/security/NoOpStreamAuthorizer.java    | 26 ---------
 .../geode/security/StreamAuthenticator.java     | 56 -------------------
 .../apache/geode/security/StreamAuthorizer.java | 19 -------
 .../geode/security/server/Authenticator.java    | 59 ++++++++++++++++++++
 .../geode/security/server/Authorizer.java       | 21 +++++++
 .../security/server/NoOpAuthenticator.java      | 48 ++++++++++++++++
 .../geode/security/server/NoOpAuthorizer.java   | 27 +++++++++
 .../apache/geode/security/server/package.html   | 45 +++++++++++++++
 ...rg.apache.geode.security.StreamAuthenticator |  1 -
 ...g.apache.geode.security.server.Authenticator |  1 +
 .../GenericProtocolServerConnectionTest.java    |  4 +-
 .../protocol/protobuf/ProtobufOpsProcessor.java |  2 -
 .../protobuf/ProtobufSimpleAuthenticator.java   | 11 ++--
 .../protobuf/ProtobufSimpleAuthorizer.java      | 12 +---
 .../protobuf/ProtobufStreamProcessor.java       |  1 -
 ...rg.apache.geode.security.StreamAuthenticator |  1 -
 ...g.apache.geode.security.server.Authenticator |  1 +
 .../ProtobufSimpleAuthenticatorJUnitTest.java   |  7 +--
 .../protobuf/ProtobufStreamProcessorTest.java   |  4 +-
 .../GetAllRequestOperationHandlerJUnitTest.java | 12 ++--
 ...onNamesRequestOperationHandlerJUnitTest.java |  6 +-
 ...tRegionRequestOperationHandlerJUnitTest.java |  6 +-
 .../GetRequestOperationHandlerJUnitTest.java    | 12 ++--
 .../PutAllRequestOperationHandlerJUnitTest.java |  8 +--
 .../PutRequestOperationHandlerJUnitTest.java    | 10 ++--
 .../RemoveRequestOperationHandlerJUnitTest.java | 10 ++--
 31 files changed, 265 insertions(+), 231 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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 0d1dfd9..0ced3aa 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,9 +15,6 @@
 
 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/ed3ac98c/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnection.java
----------------------------------------------------------------------
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 0b53cb7..6c81028 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
@@ -20,7 +20,7 @@ import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.security.SecurityManager;
-import org.apache.geode.security.StreamAuthenticator;
+import org.apache.geode.security.server.Authenticator;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -35,7 +35,7 @@ public class GenericProtocolServerConnection extends ServerConnection {
   // The new protocol lives in a separate module and gets loaded when this class is instantiated.
   private final ClientProtocolMessageHandler messageHandler;
   private final SecurityManager securityManager;
-  private final StreamAuthenticator authenticator;
+  private final Authenticator authenticator;
 
   /**
    * Creates a new <code>GenericProtocolServerConnection</code> that processes messages received
@@ -44,7 +44,7 @@ public class GenericProtocolServerConnection extends ServerConnection {
   public GenericProtocolServerConnection(Socket s, InternalCache c, CachedRegionHelper helper,
       CacheServerStats stats, int hsTimeout, int socketBufferSize, String communicationModeStr,
       byte communicationMode, Acceptor acceptor, ClientProtocolMessageHandler newClientProtocol,
-      SecurityService securityService, StreamAuthenticator authenticator) {
+      SecurityService securityService, Authenticator authenticator) {
     super(s, c, helper, stats, hsTimeout, socketBufferSize, communicationModeStr, communicationMode,
         acceptor, securityService);
     securityManager = securityService.getSecurityManager();
@@ -60,7 +60,7 @@ public class GenericProtocolServerConnection extends ServerConnection {
       OutputStream outputStream = socket.getOutputStream();
 
       if (!authenticator.isAuthenticated()) {
-        authenticator.receiveMessage(inputStream, outputStream, securityManager);
+        authenticator.authenticate(inputStream, outputStream, securityManager);
       } else {
         messageHandler.receiveMessage(inputStream, outputStream,
             new MessageExecutionContext(this.getCache(), authenticator.getAuthorizer()));

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
index 817df0e..06d3b03 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/MessageExecutionContext.java
@@ -18,16 +18,16 @@ package org.apache.geode.internal.cache.tier.sockets;
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.security.NoOpStreamAuthorizer;
-import org.apache.geode.security.StreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
+import org.apache.geode.security.server.Authorizer;
 
 @Experimental
 public class MessageExecutionContext {
   private Cache cache;
   private InternalLocator locator;
-  private StreamAuthorizer authorizer;
+  private Authorizer authorizer;
 
-  public MessageExecutionContext(Cache cache, StreamAuthorizer streamAuthorizer) {
+  public MessageExecutionContext(Cache cache, Authorizer streamAuthorizer) {
     this.cache = cache;
     this.authorizer = streamAuthorizer;
   }
@@ -36,7 +36,7 @@ public class MessageExecutionContext {
     this.locator = locator;
     // set a no-op authorizer until such time as locators implement authentication
     // and authorization checks
-    this.authorizer = new NoOpStreamAuthorizer();
+    this.authorizer = new NoOpAuthorizer();
   }
 
   /**
@@ -68,10 +68,10 @@ public class MessageExecutionContext {
   }
 
   /**
-   * Returns the StreamAuthorizer associated with this execution. This can be used to perform
+   * Returns the Authorizer associated with this execution. This can be used to perform
    * authorization checks for the user associated with this thread.
    */
-  public StreamAuthorizer getAuthorizer() {
+  public Authorizer getAuthorizer() {
     return authorizer;
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionFactory.java
index d2d85f6..00e8b88 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
@@ -19,7 +19,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.StreamAuthenticator;
+import org.apache.geode.security.server.Authenticator;
 
 import java.io.IOException;
 import java.net.Socket;
@@ -32,7 +32,7 @@ import java.util.ServiceLoader;
  */
 public class ServerConnectionFactory {
   private ClientProtocolMessageHandler protocolHandler;
-  private Map<String, Class<? extends StreamAuthenticator>> authenticators = null;
+  private Map<String, Class<? extends Authenticator>> authenticators = null;
 
   public ServerConnectionFactory() {}
 
@@ -41,8 +41,8 @@ public class ServerConnectionFactory {
       return;
     }
     authenticators = new HashMap<>();
-    ServiceLoader<StreamAuthenticator> loader = ServiceLoader.load(StreamAuthenticator.class);
-    for (StreamAuthenticator streamAuthenticator : loader) {
+    ServiceLoader<Authenticator> loader = ServiceLoader.load(Authenticator.class);
+    for (Authenticator streamAuthenticator : loader) {
       authenticators.put(streamAuthenticator.implementationID(), streamAuthenticator.getClass());
     }
   }
@@ -57,15 +57,14 @@ public class ServerConnectionFactory {
     return protocolHandler;
   }
 
-  private StreamAuthenticator findStreamAuthenticator(String implementationID) {
+  private Authenticator findStreamAuthenticator(String implementationID) {
     if (authenticators == null) {
       initializeAuthenticatorsMap();
     }
-    Class<? extends StreamAuthenticator> streamAuthenticatorClass =
-        authenticators.get(implementationID);
+    Class<? extends Authenticator> streamAuthenticatorClass = authenticators.get(implementationID);
     if (streamAuthenticatorClass == null) {
       throw new ServiceLoadingFailureException(
-          "Could not find implementation for StreamAuthenticator with implementation ID "
+          "Could not find implementation for Authenticator with implementation ID "
               + implementationID);
     } else {
       try {

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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
deleted file mode 100644
index 62f49fe..0000000
--- a/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthenticator.java
+++ /dev/null
@@ -1,46 +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.security;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * An implementation of {@link StreamAuthenticator} that doesn't use its parameters and always
- * returns true.
- */
-public class NoOpStreamAuthenticator implements StreamAuthenticator {
-  @Override
-  public void receiveMessage(InputStream inputStream, OutputStream outputStream,
-      SecurityManager securityManager) throws IOException {
-    // this method needs to do nothing as it is a pass-through implementation
-  }
-
-  @Override
-  public boolean isAuthenticated() {
-    return true;
-  }
-
-  @Override
-  public StreamAuthorizer getAuthorizer() {
-    return new NoOpStreamAuthorizer();
-  }
-
-  @Override
-  public String implementationID() {
-    return "NOOP";
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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
deleted file mode 100644
index 1b21576..0000000
--- a/geode-core/src/main/java/org/apache/geode/security/NoOpStreamAuthorizer.java
+++ /dev/null
@@ -1,26 +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.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/ed3ac98c/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
deleted file mode 100644
index 8bba60c..0000000
--- a/geode-core/src/main/java/org/apache/geode/security/StreamAuthenticator.java
+++ /dev/null
@@ -1,56 +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.security;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * Implementers of this interface do some message passing over a socket to authenticate a client,
- * then hand off the connection to the protocol that will talk on the socket.
- *
- * If authentication fails, an implementor may continue to wait for another valid authentication
- * exchange.
- */
-public interface StreamAuthenticator {
-  /**
-   *
-   * @param inputStream to read auth messages from.
-   * @param outputStream to send messages to.
-   * @param securityManager can be used for validating credentials against.
-   * @throws IOException if EOF or if invalid input is received.
-   */
-  void receiveMessage(InputStream inputStream, OutputStream outputStream,
-      SecurityManager securityManager) throws IOException;
-
-  /**
-   * Until authentication is complete, isAuthenticated() must return false, and the socket will
-   * always be passed to the StreamAuthenticator. Once authentication succeeds, calls to this
-   * function must always return true.
-   */
-  boolean isAuthenticated();
-
-  /**
-   * Return an authorization object which can be used to determine which permissions this stream has
-   * according to the provided securityManager.
-   */
-  StreamAuthorizer getAuthorizer() throws AuthenticationRequiredException;
-
-  /**
-   * @return a unique identifier for this particular implementation (NOOP, PASSTHROUGH, etc.)
-   */
-  String implementationID();
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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
deleted file mode 100644
index fdb6b17..0000000
--- a/geode-core/src/main/java/org/apache/geode/security/StreamAuthorizer.java
+++ /dev/null
@@ -1,19 +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.security;
-
-public interface StreamAuthorizer {
-  boolean authorize(ResourcePermission permissionRequested);
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/security/server/Authenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/Authenticator.java b/geode-core/src/main/java/org/apache/geode/security/server/Authenticator.java
new file mode 100644
index 0000000..7893b4b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/server/Authenticator.java
@@ -0,0 +1,59 @@
+/*
+ * 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.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.security.SecurityManager;
+
+/**
+ * Implementers of this interface do some message passing over a socket to authenticate a client,
+ * then hand off the connection to the protocol that will talk on the socket.
+ *
+ * If authentication fails, an implementor may continue to wait for another valid authentication
+ * exchange.
+ */
+public interface Authenticator {
+  /**
+   *
+   * @param inputStream to read auth messages from.
+   * @param outputStream to send messages to.
+   * @param securityManager can be used for validating credentials against.
+   * @throws IOException if EOF or if invalid input is received.
+   */
+  void authenticate(InputStream inputStream, OutputStream outputStream,
+      SecurityManager securityManager) throws IOException;
+
+  /**
+   * Until authentication is complete, isAuthenticated() must return false, and the socket will
+   * always be passed to the Authenticator. Once authentication succeeds, calls to this function
+   * must always return true.
+   */
+  boolean isAuthenticated();
+
+  /**
+   * Return an authorization object which can be used to determine which permissions this stream has
+   * according to the provided securityManager.
+   */
+  Authorizer getAuthorizer() throws AuthenticationRequiredException;
+
+  /**
+   * @return a unique identifier for this particular implementation (NOOP, PASSTHROUGH, etc.)
+   */
+  String implementationID();
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/security/server/Authorizer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/Authorizer.java b/geode-core/src/main/java/org/apache/geode/security/server/Authorizer.java
new file mode 100644
index 0000000..fea2198
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/server/Authorizer.java
@@ -0,0 +1,21 @@
+/*
+ * 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.server;
+
+import org.apache.geode.security.ResourcePermission;
+
+public interface Authorizer {
+  boolean authorize(ResourcePermission permissionRequested);
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java b/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
new file mode 100644
index 0000000..bf435d2
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.security.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.geode.security.SecurityManager;
+
+/**
+ * An implementation of {@link Authenticator} that doesn't use its parameters and always returns
+ * true.
+ */
+public class NoOpAuthenticator implements Authenticator {
+  @Override
+  public void authenticate(InputStream inputStream, OutputStream outputStream,
+      SecurityManager securityManager) throws IOException {
+    // this method needs to do nothing as it is a pass-through implementation
+  }
+
+  @Override
+  public boolean isAuthenticated() {
+    return true;
+  }
+
+  @Override
+  public Authorizer getAuthorizer() {
+    return new NoOpAuthorizer();
+  }
+
+  @Override
+  public String implementationID() {
+    return "NOOP";
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/java/org/apache/geode/security/server/package.html
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/package.html b/geode-core/src/main/java/org/apache/geode/security/server/package.html
new file mode 100644
index 0000000..15a2114
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/server/package.html
@@ -0,0 +1,45 @@
+<!--
+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.
+-->
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="UTF-8">
+</head>
+<body>
+<p>This package provides authentication and authorization for a client.  There is a default
+"no-op" implementation that is overridden by the Protobuf form of client/server
+   communication.</p>
+<p>This can also be overridden by _you_ in order to use a different mechanism
+on the server by setting the property geode.protocol-authentication-mode to the name
+   of your implementation.  This is the string returned by your class's
+   implementationID method.</p>
+
+<p>The Authenticator.authenticate() method
+   is first given the i/o streams connected to the client so that it
+can interact with the client to get credentials.  A simple mechanism might expect a
+username and password on the input stream.</p>
+
+<p>Once the client is authenticated the Authenticator will be asked for an authorizer each time
+a message is received.  The authorizer will be asked if the authenticated principal
+is authorized for permissions associated with the request sent by the client.</p>
+
+<p><i>Note: the legacy "GemFire" client/server authorization mechanism is hardcoded
+to get credentials using a non-documented interchange and is not affected by this package.
+It interacts directly with the SecurityManager, as does the default mechanism used in
+Protobuf communications.</i></p>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
----------------------------------------------------------------------
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator b/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
deleted file mode 100644
index 3b93815..0000000
--- a/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.geode.security.NoOpStreamAuthenticator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
----------------------------------------------------------------------
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator b/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
new file mode 100644
index 0000000..4f34d2a
--- /dev/null
+++ b/geode-core/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
@@ -0,0 +1 @@
+org.apache.geode.security.server.NoOpAuthenticator

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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..ea00018 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
@@ -24,7 +24,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.security.NoOpStreamAuthenticator;
+import org.apache.geode.security.server.NoOpAuthenticator;
 import org.apache.geode.test.junit.categories.UnitTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -58,6 +58,6 @@ public class GenericProtocolServerConnectionTest {
     return new GenericProtocolServerConnection(socketMock, mock(InternalCache.class),
         mock(CachedRegionHelper.class), mock(CacheServerStats.class), 0, 0, "",
         Acceptor.PROTOBUF_CLIENT_SERVER_PROTOCOL, mock(AcceptorImpl.class), clientProtocolMock,
-        mock(SecurityService.class), new NoOpStreamAuthenticator());
+        mock(SecurityService.class), new NoOpAuthenticator());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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 a2f752d..b95b405 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
@@ -15,13 +15,11 @@
 package org.apache.geode.protocol.protobuf;
 
 import org.apache.geode.annotations.Experimental;
-import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.tier.sockets.MessageExecutionContext;
 import org.apache.geode.internal.cache.tier.sockets.InvalidExecutionContextException;
 import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 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;
 
 /**

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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 1bb0678..5fadadd 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
@@ -17,10 +17,10 @@ package org.apache.geode.protocol.protobuf;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.internal.protocol.protobuf.AuthenticationAPI;
 import org.apache.geode.security.AuthenticationRequiredException;
-import org.apache.geode.security.StreamAuthenticator;
+import org.apache.geode.security.server.Authenticator;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.SecurityManager;
-import org.apache.geode.security.StreamAuthorizer;
+import org.apache.geode.security.server.Authorizer;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -28,12 +28,11 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Properties;
 
-public class ProtobufSimpleAuthenticator implements StreamAuthenticator {
-  private boolean authenticated;
+public class ProtobufSimpleAuthenticator implements Authenticator {
   private ProtobufSimpleAuthorizer authorizer = null;
 
   @Override
-  public void receiveMessage(InputStream inputStream, OutputStream outputStream,
+  public void authenticate(InputStream inputStream, OutputStream outputStream,
       SecurityManager securityManager) throws IOException {
     AuthenticationAPI.SimpleAuthenticationRequest authenticationRequest =
         AuthenticationAPI.SimpleAuthenticationRequest.parseDelimitedFrom(inputStream);
@@ -66,7 +65,7 @@ public class ProtobufSimpleAuthenticator implements StreamAuthenticator {
   }
 
   @Override
-  public StreamAuthorizer getAuthorizer() throws AuthenticationRequiredException {
+  public Authorizer getAuthorizer() throws AuthenticationRequiredException {
     if (authorizer == null) {
       throw new AuthenticationRequiredException("Not yet authenticated");
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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
index 872632a..e29abfa 100644
--- 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
@@ -14,19 +14,11 @@
  */
 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;
+import org.apache.geode.security.server.Authorizer;
 
-public class ProtobufSimpleAuthorizer implements StreamAuthorizer {
+public class ProtobufSimpleAuthorizer implements Authorizer {
   private final Object authenticatedPrincipal;
   private final SecurityManager securityManager;
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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 23924db..f28c310 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
@@ -27,7 +27,6 @@ import org.apache.geode.internal.protocol.protobuf.ClientProtocol;
 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/ed3ac98c/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
deleted file mode 100644
index 45e4eea..0000000
--- a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.StreamAuthenticator
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.geode.protocol.protobuf.ProtobufSimpleAuthenticator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
new file mode 100644
index 0000000..45e4eea
--- /dev/null
+++ b/geode-protobuf/src/main/resources/META-INF/services/org.apache.geode.security.server.Authenticator
@@ -0,0 +1 @@
+org.apache.geode.protocol.protobuf.ProtobufSimpleAuthenticator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticatorJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticatorJUnitTest.java
index c80b7c8..33424a3 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticatorJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/ProtobufSimpleAuthenticatorJUnitTest.java
@@ -19,7 +19,6 @@ import org.apache.geode.examples.security.ExampleSecurityManager;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.internal.protocol.protobuf.AuthenticationAPI;
 import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.SecurityManager;
 import org.apache.geode.test.junit.categories.UnitTest;
 import org.junit.Before;
@@ -77,7 +76,7 @@ public class ProtobufSimpleAuthenticatorJUnitTest {
   public void successfulAuthentication() throws IOException {
     assertFalse(protobufSimpleAuthenticator.isAuthenticated());
 
-    protobufSimpleAuthenticator.receiveMessage(byteArrayInputStream, byteArrayOutputStream,
+    protobufSimpleAuthenticator.authenticate(byteArrayInputStream, byteArrayOutputStream,
         mockSecurityManager);
 
     AuthenticationAPI.SimpleAuthenticationResponse simpleAuthenticationResponse =
@@ -97,7 +96,7 @@ public class ProtobufSimpleAuthenticatorJUnitTest {
     when(mockSecurityManager.authenticate(expectedAuthProperties))
         .thenThrow(new AuthenticationFailedException("BOOM!"));
 
-    protobufSimpleAuthenticator.receiveMessage(byteArrayInputStream, byteArrayOutputStream,
+    protobufSimpleAuthenticator.authenticate(byteArrayInputStream, byteArrayOutputStream,
         mockSecurityManager);
 
     AuthenticationAPI.SimpleAuthenticationResponse simpleAuthenticationResponse =
@@ -111,7 +110,7 @@ public class ProtobufSimpleAuthenticatorJUnitTest {
   public void testExampleSecurityManager() throws IOException {
     assertFalse(protobufSimpleAuthenticator.isAuthenticated());
 
-    protobufSimpleAuthenticator.receiveMessage(byteArrayInputStream, byteArrayOutputStream,
+    protobufSimpleAuthenticator.authenticate(byteArrayInputStream, byteArrayOutputStream,
         mockSecurityManager);
 
     new ExampleSecurityManager().init(expectedAuthProperties);

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/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 bebbfde..50d7b40 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,7 +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.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -40,6 +40,6 @@ public class ProtobufStreamProcessorTest {
     ProtobufStreamProcessor protobufStreamProcessor = new ProtobufStreamProcessor();
     InternalCache mockInternalCache = mock(InternalCache.class);
     protobufStreamProcessor.receiveMessage(inputStream, outputStream,
-        new MessageExecutionContext(mockInternalCache, new NoOpStreamAuthorizer()));
+        new MessageExecutionContext(mockInternalCache, new NoOpAuthorizer()));
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
index b8613b6..f4d9c92 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetAllRequestOperationHandlerJUnitTest.java
@@ -35,7 +35,7 @@ import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -79,9 +79,8 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   public void processReturnsExpectedValuesForValidKeys()
       throws CodecAlreadyRegisteredForTypeException, UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
-    Result<RegionAPI.GetAllResponse> result =
-        operationHandler.process(serializationServiceStub, generateTestRequest(true),
-            new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+    Result<RegionAPI.GetAllResponse> result = operationHandler.process(serializationServiceStub,
+        generateTestRequest(true), new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(result instanceof Success);
 
@@ -100,9 +99,8 @@ public class GetAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
   @Test
   public void processReturnsNoEntriesForNoKeysRequested() throws UnsupportedEncodingTypeException,
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
-    Result<RegionAPI.GetAllResponse> result =
-        operationHandler.process(serializationServiceStub, generateTestRequest(false),
-            new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+    Result<RegionAPI.GetAllResponse> result = operationHandler.process(serializationServiceStub,
+        generateTestRequest(false), new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(result instanceof Success);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
index 323048e..8566fd6 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionNamesRequestOperationHandlerJUnitTest.java
@@ -33,7 +33,7 @@ import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -68,7 +68,7 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     Result<RegionAPI.GetRegionNamesResponse> result = operationHandler.process(
         serializationServiceStub, ProtobufRequestUtilities.createGetRegionNamesRequest(),
-        new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
     Assert.assertTrue(result instanceof Success);
 
     RegionAPI.GetRegionNamesResponse getRegionsResponse = result.getMessage();
@@ -94,7 +94,7 @@ public class GetRegionNamesRequestOperationHandlerJUnitTest extends OperationHan
         .thenReturn(Collections.unmodifiableSet(new HashSet<Region<String, String>>()));
     Result<RegionAPI.GetRegionNamesResponse> result = operationHandler.process(
         serializationServiceStub, ProtobufRequestUtilities.createGetRegionNamesRequest(),
-        new MessageExecutionContext(emptyCache, new NoOpStreamAuthorizer()));
+        new MessageExecutionContext(emptyCache, new NoOpAuthorizer()));
     Assert.assertTrue(result instanceof Success);
 
     RegionAPI.GetRegionNamesResponse getRegionsResponse = result.getMessage();

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
index 88950b7..0ba0fee 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRegionRequestOperationHandlerJUnitTest.java
@@ -28,7 +28,7 @@ import org.apache.geode.protocol.protobuf.Failure;
 import org.apache.geode.protocol.protobuf.ProtocolErrorCode;
 import org.apache.geode.internal.protocol.protobuf.RegionAPI;
 import org.apache.geode.protocol.protobuf.Result;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -77,7 +77,7 @@ public class GetRegionRequestOperationHandlerJUnitTest extends OperationHandlerJ
 
     Result<RegionAPI.GetRegionResponse> result = operationHandler.process(serializationServiceStub,
         MessageUtil.makeGetRegionRequest(TEST_REGION1),
-        new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
     RegionAPI.GetRegionResponse response = result.getMessage();
     BasicTypes.Region region = response.getRegion();
     Assert.assertEquals(TEST_REGION1, region.getName());
@@ -103,7 +103,7 @@ public class GetRegionRequestOperationHandlerJUnitTest extends OperationHandlerJ
     String unknownRegionName = "UNKNOWN_REGION";
     Result<RegionAPI.GetRegionResponse> result = operationHandler.process(serializationServiceStub,
         MessageUtil.makeGetRegionRequest(unknownRegionName),
-        new MessageExecutionContext(emptyCache, new NoOpStreamAuthorizer()));
+        new MessageExecutionContext(emptyCache, new NoOpAuthorizer()));
     Assert.assertTrue(result instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
         result.getErrorMessage().getError().getErrorCode());

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
index f104fb0..58945dd 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/GetRequestOperationHandlerJUnitTest.java
@@ -26,7 +26,7 @@ import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -73,7 +73,7 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, false, false);
     Result<RegionAPI.GetResponse> result = operationHandler.process(serializationServiceStub,
-        getRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        getRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(result instanceof Success);
     Assert.assertEquals(BasicTypes.EncodedValue.ValueCase.STRINGRESULT,
@@ -88,7 +88,7 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.GetRequest getRequest = generateTestRequest(true, false, false);
     Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
-        getRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        getRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(response instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -101,7 +101,7 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, true, false);
     Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
-        getRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        getRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(response instanceof Success);
   }
@@ -112,7 +112,7 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.GetRequest getRequest = generateTestRequest(false, false, true);
     Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
-        getRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        getRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(response instanceof Success);
   }
@@ -134,7 +134,7 @@ public class GetRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
     RegionAPI.GetRequest getRequest =
         ProtobufRequestUtilities.createGetRequest(TEST_REGION, encodedKey).getGetRequest();
     Result<RegionAPI.GetResponse> response = operationHandler.process(serializationServiceStub,
-        getRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        getRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(response instanceof Failure);
     Assert.assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
index 71d6a51..4958a40 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutAllRequestOperationHandlerJUnitTest.java
@@ -23,7 +23,7 @@ import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -76,7 +76,7 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
 
     Result<RegionAPI.PutAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(false, true),
-            new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+            new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     Assert.assertTrue(result instanceof Success);
 
@@ -91,7 +91,7 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
 
     Result<RegionAPI.PutAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(true, true),
-            new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+            new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Success);
     verify(regionMock).put(TEST_KEY1, TEST_VALUE1);
@@ -111,7 +111,7 @@ public class PutAllRequestOperationHandlerJUnitTest extends OperationHandlerJUni
 
     Result<RegionAPI.PutAllResponse> result =
         operationHandler.process(serializationServiceStub, generateTestRequest(false, false),
-            new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+            new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Success);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
index cbd897a..c2b9af3 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/PutRequestOperationHandlerJUnitTest.java
@@ -26,7 +26,7 @@ import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -69,7 +69,7 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
       CodecAlreadyRegisteredForTypeException, InvalidExecutionContextException {
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
     Result<RegionAPI.PutResponse> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Success);
 
@@ -100,7 +100,7 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
     RegionAPI.PutRequest putRequest =
         ProtobufRequestUtilities.createPutRequest(TEST_REGION, testEntry).getPutRequest();
     Result<RegionAPI.PutResponse> result = operationHandler.process(serializationServiceStub,
-        putRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        putRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,
@@ -114,7 +114,7 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
     when(cacheStub.getRegion(TEST_REGION)).thenReturn(null);
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
     Result<RegionAPI.PutResponse> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -129,7 +129,7 @@ public class PutRequestOperationHandlerJUnitTest extends OperationHandlerJUnitTe
 
     PutRequestOperationHandler operationHandler = new PutRequestOperationHandler();
     Result<RegionAPI.PutResponse> result = operationHandler.process(serializationServiceStub,
-        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        generateTestRequest(), new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.CONSTRAINT_VIOLATION.codeValue,

http://git-wip-us.apache.org/repos/asf/geode/blob/ed3ac98c/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
index 5d38e61..5d8a31c 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/protobuf/operations/RemoveRequestOperationHandlerJUnitTest.java
@@ -27,7 +27,7 @@ import org.apache.geode.protocol.protobuf.Result;
 import org.apache.geode.protocol.protobuf.Success;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufRequestUtilities;
 import org.apache.geode.protocol.protobuf.utilities.ProtobufUtilities;
-import org.apache.geode.security.NoOpStreamAuthorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.exception.UnsupportedEncodingTypeException;
 import org.apache.geode.serialization.registry.exception.CodecAlreadyRegisteredForTypeException;
 import org.apache.geode.serialization.registry.exception.CodecNotRegisteredForTypeException;
@@ -74,7 +74,7 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(false, false).getRemoveRequest();
     Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
-        removeRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        removeRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Success);
     verify(regionStub).remove(TEST_KEY);
@@ -86,7 +86,7 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(true, false).getRemoveRequest();
     Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
-        removeRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        removeRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.REGION_NOT_FOUND.codeValue,
@@ -99,7 +99,7 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
       CodecNotRegisteredForTypeException, InvalidExecutionContextException {
     RegionAPI.RemoveRequest removeRequest = generateTestRequest(false, true).getRemoveRequest();
     Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
-        removeRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        removeRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Success);
   }
@@ -122,7 +122,7 @@ public class RemoveRequestOperationHandlerJUnitTest extends OperationHandlerJUni
     RegionAPI.RemoveRequest removeRequest =
         ProtobufRequestUtilities.createRemoveRequest(TEST_REGION, encodedKey).getRemoveRequest();;
     Result<RegionAPI.RemoveResponse> result = operationHandler.process(serializationServiceStub,
-        removeRequest, new MessageExecutionContext(cacheStub, new NoOpStreamAuthorizer()));
+        removeRequest, new MessageExecutionContext(cacheStub, new NoOpAuthorizer()));
 
     assertTrue(result instanceof Failure);
     assertEquals(ProtocolErrorCode.VALUE_ENCODING_ERROR.codeValue,