You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2017/06/19 15:23:52 UTC

[19/24] geode git commit: GEODE-2632: consolidate different types of SecurityService

GEODE-2632: consolidate different types of SecurityService

* combine EnabledSecurityService and CustomSecurityService into IntegratedSecurityService
* combine DisabledSecurityService and LegacySecurityService
* combine ConfigInitializer and RealmInitializer
* provide default impelementations of SecurityService
* consolidate SecurityService creation.


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

Branch: refs/heads/feature/GEODE-3071
Commit: 5546a8732203baec09fd3922a8e4cb86e5048492
Parents: dd90c71
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Jun 9 12:29:10 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Jun 15 11:32:30 2017 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     |  62 +--
 .../geode/internal/cache/GemFireCacheImpl.java  |   2 -
 .../internal/security/CallbackInstantiator.java |  28 ++
 .../security/CustomSecurityService.java         | 346 ----------------
 .../security/DisabledSecurityService.java       | 221 -----------
 .../security/EnabledSecurityService.java        | 393 -------------------
 .../security/IntegratedSecurityService.java     | 374 ++++++++++++++++++
 .../security/LegacySecurityService.java         | 186 +--------
 .../internal/security/SecurityService.java      |  99 +++--
 .../security/SecurityServiceFactory.java        | 188 +++------
 .../internal/security/SecurityServiceType.java  |  28 --
 .../security/shiro/ConfigInitializer.java       |  43 --
 .../security/shiro/CustomAuthRealm.java         |  19 +-
 .../security/shiro/RealmInitializer.java        |  54 ---
 .../security/shiro/SecurityManagerProvider.java |  83 ++++
 .../membership/MembershipJUnitTest.java         |   3 -
 .../security/DisabledSecurityServiceTest.java   | 164 --------
 .../security/EnabledSecurityServiceTest.java    | 184 ---------
 .../internal/security/FakePostProcessor.java    | 103 -----
 .../internal/security/FakeSecurityManager.java  | 103 -----
 ...ntegratedSecurityServiceConstructorTest.java |  93 +++++
 .../security/IntegratedSecurityServiceTest.java | 166 ++++++++
 .../security/LegacySecurityServiceTest.java     |  58 +++
 ...urityServiceFactoryShiroIntegrationTest.java |  38 +-
 .../security/SecurityServiceFactoryTest.java    | 314 ++++++---------
 .../internal/security/SecurityServiceTest.java  |  21 +-
 .../shiro/ConfigInitializerIntegrationTest.java |  91 -----
 ...tyServiceWithCustomRealmIntegrationTest.java |   2 +-
 ...urityServiceWithShiroIniIntegrationTest.java |   2 +-
 .../CacheFactoryWithSecurityObjectTest.java     |  68 +++-
 ...SecurityManagerLifecycleDistributedTest.java |   4 +-
 .../apache/geode/tools/pulse/tests/Server.java  |  19 +-
 .../tools/pulse/tests/rules/ServerRule.java     |   6 +-
 33 files changed, 1161 insertions(+), 2404 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 22edb6f..85f9146 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -15,35 +15,8 @@
 
 package org.apache.geode.distributed.internal;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.Reader;
-import java.lang.reflect.Array;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.StringTokenizer;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.geode.cache.CacheXmlException;
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.security.SecurityServiceFactory;
-import org.apache.geode.security.PostProcessor;
-import org.apache.logging.log4j.Logger;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
@@ -60,6 +33,7 @@ import org.apache.geode.SystemFailure;
 import org.apache.geode.admin.AlertLevel;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.CacheXmlException;
 import org.apache.geode.cache.execute.internal.FunctionServiceManager;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.DistributedMember;
@@ -100,6 +74,8 @@ import org.apache.geode.internal.logging.log4j.LogWriterAppenders;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.offheap.OffHeapStorage;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.geode.internal.statistics.DummyStatisticsImpl;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
 import org.apache.geode.internal.statistics.LocalStatisticsImpl;
@@ -111,7 +87,31 @@ import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.management.ManagementException;
 import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.PostProcessor;
 import org.apache.geode.security.SecurityManager;
+import org.apache.logging.log4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.reflect.Array;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.StringTokenizer;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * The concrete implementation of {@link DistributedSystem} that provides internal-only
@@ -586,8 +586,8 @@ public class InternalDistributedSystem extends DistributedSystem
 
     this.config = new RuntimeDistributionConfigImpl(this);
 
-    this.securityService =
-        SecurityServiceFactory.create(this.config, securityManager, postProcessor);
+    this.securityService = SecurityServiceFactory.create(this.config.getSecurityProps(),
+        securityManager, postProcessor);
 
     if (!this.isLoner) {
       this.attemptingToReconnect = (reconnectAttemptCounter > 0);

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 40df0c7..5eaa5a4 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -126,7 +126,6 @@ import org.apache.geode.cache.client.PoolFactory;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.client.internal.ClientMetadataService;
 import org.apache.geode.cache.client.internal.ClientRegionFactoryImpl;
-import org.apache.geode.cache.client.internal.ConnectionImpl;
 import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.client.internal.PoolImpl;
 import org.apache.geode.cache.control.ResourceManager;
@@ -213,7 +212,6 @@ import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.process.ClusterConfigurationNotAvailableException;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.geode.internal.sequencelog.SequenceLoggerImpl;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.internal.util.concurrent.FutureResult;

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/CallbackInstantiator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/CallbackInstantiator.java b/geode-core/src/main/java/org/apache/geode/internal/security/CallbackInstantiator.java
index 3ff632d..2cb8d08 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/CallbackInstantiator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/CallbackInstantiator.java
@@ -14,12 +14,18 @@
  */
 package org.apache.geode.internal.security;
 
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
 import static org.apache.geode.internal.ClassLoadUtil.classFromName;
 import static org.apache.geode.internal.ClassLoadUtil.methodFromName;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.PostProcessor;
+import org.apache.geode.security.SecurityManager;
 
 import java.lang.reflect.Method;
+import java.util.Properties;
 
 /**
  * Utility methods for instantiating security callback objects by reflection.
@@ -50,6 +56,28 @@ public class CallbackInstantiator {
     }
   }
 
+  public static SecurityManager getSecurityManager(Properties properties) {
+    String securityManagerConfig = properties.getProperty(SECURITY_MANAGER);
+    if (StringUtils.isBlank(securityManagerConfig)) {
+      return null;
+    }
+    SecurityManager securityManager =
+        getObjectOfTypeFromClassName(securityManagerConfig, SecurityManager.class);
+    securityManager.init(properties);
+    return securityManager;
+  }
+
+  public static PostProcessor getPostProcessor(Properties properties) {
+    String postProcessorConfig = properties.getProperty(SECURITY_POST_PROCESSOR);
+    if (StringUtils.isBlank(postProcessorConfig)) {
+      return null;
+    }
+    PostProcessor postProcessor =
+        getObjectOfTypeFromClassName(postProcessorConfig, PostProcessor.class);
+    postProcessor.init(properties);
+    return postProcessor;
+  }
+
   /**
    * this method would never return null, it either throws an exception or returns an object
    */

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/CustomSecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/CustomSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/CustomSecurityService.java
deleted file mode 100644
index 0ba1cb6..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/CustomSecurityService.java
+++ /dev/null
@@ -1,346 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security;
-
-import org.apache.commons.lang.SerializationException;
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.GemFireIOException;
-import org.apache.geode.internal.cache.EntryEventImpl;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.security.shiro.GeodeAuthenticationToken;
-import org.apache.geode.internal.security.shiro.ShiroPrincipal;
-import org.apache.geode.internal.util.BlobHelper;
-import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.PostProcessor;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.security.ResourcePermission.Target;
-import org.apache.geode.security.SecurityManager;
-import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.ShiroException;
-import org.apache.shiro.subject.Subject;
-import org.apache.shiro.subject.support.SubjectThreadState;
-import org.apache.shiro.util.ThreadContext;
-import org.apache.shiro.util.ThreadState;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessController;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-public class CustomSecurityService implements SecurityService {
-  private static Logger logger = LogService.getLogger(LogService.SECURITY_LOGGER_NAME);
-
-  private final PostProcessor postProcessor;
-
-  CustomSecurityService(PostProcessor postProcessor) {
-    this.postProcessor = postProcessor;
-  }
-
-  @Override
-  public void initSecurity(final Properties securityProps) {
-    if (this.postProcessor != null) {
-      this.postProcessor.init(securityProps);
-    }
-  }
-
-  @Override
-  public ThreadState bindSubject(final Subject subject) {
-    if (subject == null) {
-      return null;
-    }
-
-    ThreadState threadState = new SubjectThreadState(subject);
-    threadState.bind();
-    return threadState;
-  }
-
-  @Override
-  public Subject getSubject() {
-    Subject currentUser;
-
-    // First try get the principal out of AccessControlContext instead of Shiro's Thread context
-    // since threads can be shared between JMX clients.
-    javax.security.auth.Subject jmxSubject =
-        javax.security.auth.Subject.getSubject(AccessController.getContext());
-
-    if (jmxSubject != null) {
-      Set<ShiroPrincipal> principals = jmxSubject.getPrincipals(ShiroPrincipal.class);
-      if (principals.size() > 0) {
-        ShiroPrincipal principal = principals.iterator().next();
-        currentUser = principal.getSubject();
-        ThreadContext.bind(currentUser);
-        return currentUser;
-      }
-    }
-
-    // in other cases like rest call, client operations, we get it from the current thread
-    currentUser = SecurityUtils.getSubject();
-
-    if (currentUser == null || currentUser.getPrincipal() == null) {
-      throw new GemFireSecurityException("Error: Anonymous User");
-    }
-
-    return currentUser;
-  }
-
-  @Override
-  public Subject login(final Properties credentials) {
-    if (credentials == null) {
-      return null;
-    }
-
-    // this makes sure it starts with a clean user object
-    ThreadContext.remove();
-
-    Subject currentUser = SecurityUtils.getSubject();
-    GeodeAuthenticationToken token = new GeodeAuthenticationToken(credentials);
-    try {
-      logger.debug("Logging in " + token.getPrincipal());
-      currentUser.login(token);
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(
-          "Authentication error. Please check your credentials.", e);
-    }
-
-    return currentUser;
-  }
-
-  @Override
-  public void logout() {
-    Subject currentUser = getSubject();
-    if (currentUser == null) {
-      return;
-    }
-
-    try {
-      logger.debug("Logging out " + currentUser.getPrincipal());
-      currentUser.logout();
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new GemFireSecurityException(e.getMessage(), e);
-    }
-    // clean out Shiro's thread local content
-    ThreadContext.remove();
-  }
-
-  @Override
-  public Callable associateWith(final Callable callable) {
-    Subject currentUser = getSubject();
-    if (currentUser == null) {
-      return callable;
-    }
-
-    return currentUser.associateWith(callable);
-  }
-
-  public void authorizeClusterManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeClusterWrite() {
-    authorize(Resource.CLUSTER, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeClusterRead() {
-    authorize(Resource.CLUSTER, Operation.READ, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataManage() {
-    authorize(Resource.DATA, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataWrite() {
-    authorize(Resource.DATA, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataRead() {
-    authorize(Resource.DATA, Operation.READ, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDiskManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.DISK, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeGatewayManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.GATEWAY, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeJarManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.JAR, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeQueryManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.QUERY, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName) {
-    authorize(Resource.DATA, Operation.MANAGE, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.MANAGE, regionName, key);
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName) {
-    authorize(Resource.DATA, Operation.WRITE, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.WRITE, regionName, key);
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName) {
-    authorize(Resource.DATA, Operation.READ, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.READ, regionName, key);
-  }
-
-  public void authorize(Resource resource, Operation operation, ResourcePermission.Target target,
-      String key) {
-    authorize(resource, operation, target.getName(), key);
-  }
-
-  public void authorize(Resource resource, Operation operation, ResourcePermission.Target target) {
-    authorize(resource, operation, target, ResourcePermission.ALL);
-  }
-
-  public void authorize(Resource resource, Operation operation, String target, String key) {
-    authorize(new ResourcePermission(resource, operation, target, key));
-  }
-
-  @Override
-  public void authorize(final ResourcePermission context) {
-    Subject currentUser = getSubject();
-    if (currentUser == null) {
-      return;
-    }
-    if (context == null) {
-      return;
-    }
-    if (context.getResource() == Resource.NULL && context.getOperation() == Operation.NULL) {
-      return;
-    }
-
-    try {
-      currentUser.checkPermission(context);
-    } catch (ShiroException e) {
-      String msg = currentUser.getPrincipal() + " not authorized for " + context;
-      logger.info(msg);
-      throw new NotAuthorizedException(msg, e);
-    }
-  }
-
-  @Override
-  public void close() {
-    ThreadContext.remove();
-    SecurityUtils.setSecurityManager(null);
-  }
-
-  @Override
-  public boolean needPostProcess() {
-    return this.postProcessor != null;
-  }
-
-  @Override
-  public Object postProcess(final String regionPath, final Object key, final Object value,
-      final boolean valueIsSerialized) {
-    return postProcess(null, regionPath, key, value, valueIsSerialized);
-  }
-
-  @Override
-  public Object postProcess(Object principal, final String regionPath, final Object key,
-      final Object value, final boolean valueIsSerialized) {
-    if (!needPostProcess()) {
-      return value;
-    }
-
-    if (principal == null) {
-      Subject subject = getSubject();
-      if (subject == null) {
-        return value;
-      }
-      principal = (Serializable) subject.getPrincipal();
-    }
-
-    String regionName = StringUtils.stripStart(regionPath, "/");
-    Object newValue;
-
-    // if the data is a byte array, but the data itself is supposed to be an object, we need to
-    // deserialize it before we pass it to the callback.
-    if (valueIsSerialized && value instanceof byte[]) {
-      try {
-        Object oldObj = EntryEventImpl.deserialize((byte[]) value);
-        Object newObj = this.postProcessor.processRegionValue(principal, regionName, key, oldObj);
-        newValue = BlobHelper.serializeToBlob(newObj);
-      } catch (IOException | SerializationException e) {
-        throw new GemFireIOException("Exception de/serializing entry value", e);
-      }
-    } else {
-      newValue = this.postProcessor.processRegionValue(principal, regionName, key, value);
-    }
-
-    return newValue;
-  }
-
-  @Override
-  public boolean isClientSecurityRequired() {
-    return true;
-  }
-
-  @Override
-  public boolean isIntegratedSecurity() {
-    return true;
-  }
-
-  @Override
-  public boolean isPeerSecurityRequired() {
-    return true;
-  }
-
-  @Override
-  public SecurityManager getSecurityManager() {
-    return null;
-  }
-
-  @Override
-  public PostProcessor getPostProcessor() {
-    return this.postProcessor;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/DisabledSecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/DisabledSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/DisabledSecurityService.java
deleted file mode 100644
index b505690..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/DisabledSecurityService.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security;
-
-import org.apache.geode.security.PostProcessor;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.SecurityManager;
-import org.apache.shiro.subject.Subject;
-import org.apache.shiro.subject.support.SubjectThreadState;
-import org.apache.shiro.util.ThreadState;
-
-import java.util.Properties;
-import java.util.concurrent.Callable;
-
-/**
- * No-op security service that does nothing.
- */
-public class DisabledSecurityService implements SecurityService {
-
-  public DisabledSecurityService() {
-    // nothing
-  }
-
-  @Override
-  public void initSecurity(final Properties securityProps) {
-    // nothing
-  }
-
-  @Override
-  public ThreadState bindSubject(final Subject subject) {
-    if (subject == null) {
-      return null;
-    }
-
-    ThreadState threadState = new SubjectThreadState(subject);
-    threadState.bind();
-    return threadState;
-  }
-
-  @Override
-  public Subject getSubject() {
-    return null;
-  }
-
-  @Override
-  public Subject login(final Properties credentials) {
-    return null;
-  }
-
-  @Override
-  public void logout() {
-    // nothing
-  }
-
-  @Override
-  public Callable associateWith(final Callable callable) {
-    return callable;
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, String target, String key) {
-
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, ResourcePermission.Target target, String key) {
-
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, ResourcePermission.Target target) {
-
-  }
-
-  @Override
-  public void authorizeClusterManage() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeClusterWrite() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeClusterRead() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataManage() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataWrite() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataRead() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDiskManage() {
-
-  }
-
-  @Override
-  public void authorizeGatewayManage() {
-
-  }
-
-  @Override
-  public void authorizeJarManage() {
-
-  }
-
-  @Override
-  public void authorizeQueryManage() {
-
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorize(final ResourcePermission context) {
-    // nothing
-  }
-
-  @Override
-  public void close() {
-    // nothing
-  }
-
-  @Override
-  public boolean needPostProcess() {
-    return false;
-  }
-
-  @Override
-  public Object postProcess(final String regionPath, final Object key, final Object value,
-      final boolean valueIsSerialized) {
-    return value;
-  }
-
-  @Override
-  public Object postProcess(final Object principal, final String regionPath, final Object key,
-      final Object value, final boolean valueIsSerialized) {
-    return value;
-  }
-
-  @Override
-  public boolean isClientSecurityRequired() {
-    return false;
-  }
-
-  @Override
-  public boolean isIntegratedSecurity() {
-    return false;
-  }
-
-  @Override
-  public boolean isPeerSecurityRequired() {
-    return false;
-  }
-
-  @Override
-  public SecurityManager getSecurityManager() {
-    return null;
-  }
-
-  @Override
-  public PostProcessor getPostProcessor() {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/EnabledSecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/EnabledSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/EnabledSecurityService.java
deleted file mode 100644
index f0568c0..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/EnabledSecurityService.java
+++ /dev/null
@@ -1,393 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security;
-
-import org.apache.commons.lang.SerializationException;
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.GemFireIOException;
-import org.apache.geode.internal.cache.EntryEventImpl;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.security.shiro.GeodeAuthenticationToken;
-import org.apache.geode.internal.security.shiro.RealmInitializer;
-import org.apache.geode.internal.security.shiro.ShiroPrincipal;
-import org.apache.geode.internal.util.BlobHelper;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.PostProcessor;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.security.ResourcePermission.Target;
-import org.apache.geode.security.SecurityManager;
-import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.ShiroException;
-import org.apache.shiro.subject.Subject;
-import org.apache.shiro.subject.support.SubjectThreadState;
-import org.apache.shiro.util.ThreadContext;
-import org.apache.shiro.util.ThreadState;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessController;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-/**
- * Security service with SecurityManager and an optional PostProcessor.
- */
-public class EnabledSecurityService implements SecurityService {
-  private static Logger logger = LogService.getLogger(LogService.SECURITY_LOGGER_NAME);
-
-  private final SecurityManager securityManager;
-
-  private final PostProcessor postProcessor;
-
-  EnabledSecurityService(final SecurityManager securityManager, final PostProcessor postProcessor,
-      final RealmInitializer realmInitializer) {
-    this.securityManager = securityManager;
-    this.postProcessor = postProcessor;
-    realmInitializer.initialize(this.securityManager);
-  }
-
-  @Override
-  public void initSecurity(final Properties securityProps) {
-    this.securityManager.init(securityProps);
-    if (this.postProcessor != null) {
-      this.postProcessor.init(securityProps);
-    }
-  }
-
-  /**
-   * It first looks the shiro subject in AccessControlContext since JMX will use multiple threads to
-   * process operations from the same client, then it looks into Shiro's thead context.
-   *
-   * @return the shiro subject, null if security is not enabled
-   */
-  @Override
-  public Subject getSubject() {
-    Subject currentUser;
-
-    // First try get the principal out of AccessControlContext instead of Shiro's Thread context
-    // since threads can be shared between JMX clients.
-    javax.security.auth.Subject jmxSubject =
-        javax.security.auth.Subject.getSubject(AccessController.getContext());
-
-    if (jmxSubject != null) {
-      Set<ShiroPrincipal> principals = jmxSubject.getPrincipals(ShiroPrincipal.class);
-      if (!principals.isEmpty()) {
-        ShiroPrincipal principal = principals.iterator().next();
-        currentUser = principal.getSubject();
-        ThreadContext.bind(currentUser);
-        return currentUser;
-      }
-    }
-
-    // in other cases like rest call, client operations, we get it from the current thread
-    currentUser = SecurityUtils.getSubject();
-
-    if (currentUser == null || currentUser.getPrincipal() == null) {
-      throw new GemFireSecurityException("Error: Anonymous User");
-    }
-
-    return currentUser;
-  }
-
-  /**
-   * @return null if security is not enabled, otherwise return a shiro subject
-   */
-  @Override
-  public Subject login(final Properties credentials) {
-    if (credentials == null) {
-      return null;
-    }
-
-    // this makes sure it starts with a clean user object
-    ThreadContext.remove();
-
-    Subject currentUser = SecurityUtils.getSubject();
-    GeodeAuthenticationToken token = new GeodeAuthenticationToken(credentials);
-    try {
-      logger.info("Logging in " + token.getPrincipal());
-      currentUser.login(token);
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(
-          "Authentication error. Please check your credentials.", e);
-    }
-
-    return currentUser;
-  }
-
-  @Override
-  public void logout() {
-    Subject currentUser = getSubject();
-    if (currentUser == null) {
-      return;
-    }
-
-    try {
-      logger.info("Logging out " + currentUser.getPrincipal());
-      currentUser.logout();
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new GemFireSecurityException(e.getMessage(), e);
-    }
-
-    // clean out Shiro's thread local content
-    ThreadContext.remove();
-  }
-
-  @Override
-  public Callable associateWith(final Callable callable) {
-    Subject currentUser = getSubject();
-    if (currentUser == null) {
-      return callable;
-    }
-
-    return currentUser.associateWith(callable);
-  }
-
-  /**
-   * Binds the passed-in subject to the executing thread. Usage:
-   *
-   * <pre>
-   * ThreadState state = null;
-   * try {
-   *   state = securityService.bindSubject(subject);
-   *   // do the rest of the work as this subject
-   * } finally {
-   *   if (state != null)
-   *     state.clear();
-   * }
-   * </pre>
-   */
-  @Override
-  public ThreadState bindSubject(final Subject subject) {
-    if (subject == null) {
-      return null;
-    }
-
-    ThreadState threadState = new SubjectThreadState(subject);
-    threadState.bind();
-    return threadState;
-  }
-
-  public void authorizeClusterManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeClusterWrite() {
-    authorize(Resource.CLUSTER, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeClusterRead() {
-    authorize(Resource.CLUSTER, Operation.READ, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataManage() {
-    authorize(Resource.DATA, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataWrite() {
-    authorize(Resource.DATA, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDataRead() {
-    authorize(Resource.DATA, Operation.READ, Target.ALL, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeDiskManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.DISK, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeGatewayManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.GATEWAY, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeJarManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.JAR, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeQueryManage() {
-    authorize(Resource.CLUSTER, Operation.MANAGE, Target.QUERY, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName) {
-    authorize(Resource.DATA, Operation.MANAGE, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.MANAGE, regionName, key);
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName) {
-    authorize(Resource.DATA, Operation.WRITE, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.WRITE, regionName, key);
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName) {
-    authorize(Resource.DATA, Operation.READ, regionName, ResourcePermission.ALL);
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName, final String key) {
-    authorize(Resource.DATA, Operation.READ, regionName, key);
-  }
-
-  public void authorize(Resource resource, Operation operation, Target target, String key) {
-    authorize(resource, operation, target.getName(), key);
-  }
-
-  public void authorize(Resource resource, Operation operation, Target target) {
-    authorize(resource, operation, target, ResourcePermission.ALL);
-  }
-
-  public void authorize(Resource resource, Operation operation, String target, String key) {
-    authorize(new ResourcePermission(resource, operation, target, key));
-  }
-
-  @Override
-  public void authorize(final ResourcePermission context) {
-    if (context == null) {
-      return;
-    }
-    if (context.getResource() == Resource.NULL && context.getOperation() == Operation.NULL) {
-      return;
-    }
-
-    // if currentUser is null, let it throw NPE, since in a EnabledSecurityService,
-    // user can not be null
-    Subject currentUser = getSubject();
-
-    try {
-      currentUser.checkPermission(context);
-    } catch (ShiroException e) {
-      String msg = currentUser.getPrincipal() + " not authorized for " + context;
-      logger.info(msg);
-      throw new NotAuthorizedException(msg, e);
-    }
-  }
-
-  @Override
-  public void close() {
-    if (this.securityManager != null) {
-      this.securityManager.close();
-    }
-
-    if (this.postProcessor != null) {
-      this.postProcessor.close();
-    }
-
-    ThreadContext.remove();
-    SecurityUtils.setSecurityManager(null);
-  }
-
-  /**
-   * postProcess call already has this logic built in, you don't need to call this everytime you
-   * call postProcess. But if your postProcess is pretty involved with preparations and you need to
-   * bypass it entirely, call this first.
-   */
-  @Override
-  public boolean needPostProcess() {
-    return this.postProcessor != null;
-  }
-
-  @Override
-  public Object postProcess(final String regionPath, final Object key, final Object value,
-      final boolean valueIsSerialized) {
-    return postProcess(null, regionPath, key, value, valueIsSerialized);
-  }
-
-  @Override
-  public Object postProcess(Object principal, final String regionPath, final Object key,
-      final Object value, final boolean valueIsSerialized) {
-    if (!needPostProcess()) {
-      return value;
-    }
-
-    if (principal == null) {
-      Subject subject = getSubject();
-      if (subject == null) {
-        return value;
-      }
-      principal = (Serializable) subject.getPrincipal();
-    }
-
-    String regionName = StringUtils.stripStart(regionPath, "/");
-    Object newValue;
-
-    // if the data is a byte array, but the data itself is supposed to be an object, we need to
-    // deserialize it before we pass it to the callback.
-    if (valueIsSerialized && value instanceof byte[]) {
-      try {
-        Object oldObj = EntryEventImpl.deserialize((byte[]) value);
-        Object newObj = this.postProcessor.processRegionValue(principal, regionName, key, oldObj);
-        newValue = BlobHelper.serializeToBlob(newObj);
-      } catch (IOException | SerializationException e) {
-        throw new GemFireIOException("Exception de/serializing entry value", e);
-      }
-    } else {
-      newValue = this.postProcessor.processRegionValue(principal, regionName, key, value);
-    }
-
-    return newValue;
-  }
-
-  @Override
-  public SecurityManager getSecurityManager() {
-    return this.securityManager;
-  }
-
-  @Override
-  public PostProcessor getPostProcessor() {
-    return this.postProcessor;
-  }
-
-  @Override
-  public boolean isIntegratedSecurity() {
-    return true;
-  }
-
-  @Override
-  public boolean isClientSecurityRequired() {
-    return true;
-  }
-
-  @Override
-  public boolean isPeerSecurityRequired() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
new file mode 100644
index 0000000..171cfb7
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.security;
+
+import java.io.IOException;
+import java.security.AccessController;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.lang.SerializationException;
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.ShiroException;
+import org.apache.shiro.subject.Subject;
+import org.apache.shiro.subject.support.SubjectThreadState;
+import org.apache.shiro.util.ThreadContext;
+import org.apache.shiro.util.ThreadState;
+
+import org.apache.geode.GemFireIOException;
+import org.apache.geode.internal.cache.EntryEventImpl;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.security.shiro.GeodeAuthenticationToken;
+import org.apache.geode.internal.security.shiro.SecurityManagerProvider;
+import org.apache.geode.internal.security.shiro.ShiroPrincipal;
+import org.apache.geode.internal.util.BlobHelper;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.AuthenticationRequiredException;
+import org.apache.geode.security.GemFireSecurityException;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.security.PostProcessor;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.geode.security.ResourcePermission.Target;
+
+/**
+ * Security service with SecurityManager and an optional PostProcessor.
+ */
+public class IntegratedSecurityService implements SecurityService {
+  private static Logger logger = LogService.getLogger(LogService.SECURITY_LOGGER_NAME);
+
+  private final PostProcessor postProcessor;
+  private final org.apache.geode.security.SecurityManager securityManager;
+
+  /**
+   * this creates a security service using a SecurityManager
+   * 
+   * @param provider this provides shiro security manager
+   * @param postProcessor this can be null
+   */
+  IntegratedSecurityService(SecurityManagerProvider provider, PostProcessor postProcessor) {
+    // provider must provide a shiro security manager, otherwise, this is not integrated security
+    // service at all.
+    assert provider.getShiroSecurityManager() != null;
+    SecurityUtils.setSecurityManager(provider.getShiroSecurityManager());
+
+    this.securityManager = provider.getSecurityManager();
+    this.postProcessor = postProcessor;
+  }
+
+  public PostProcessor getPostProcessor() {
+    return this.postProcessor;
+  }
+
+  public org.apache.geode.security.SecurityManager getSecurityManager() {
+    return this.securityManager;
+  }
+
+  /**
+   * It first looks the shiro subject in AccessControlContext since JMX will use multiple threads to
+   * process operations from the same client, then it looks into Shiro's thead context.
+   *
+   * @return the shiro subject, null if security is not enabled
+   */
+  @Override
+  public Subject getSubject() {
+    Subject currentUser;
+
+    // First try get the principal out of AccessControlContext instead of Shiro's Thread context
+    // since threads can be shared between JMX clients.
+    javax.security.auth.Subject jmxSubject =
+        javax.security.auth.Subject.getSubject(AccessController.getContext());
+
+    if (jmxSubject != null) {
+      Set<ShiroPrincipal> principals = jmxSubject.getPrincipals(ShiroPrincipal.class);
+      if (!principals.isEmpty()) {
+        ShiroPrincipal principal = principals.iterator().next();
+        currentUser = principal.getSubject();
+        ThreadContext.bind(currentUser);
+        return currentUser;
+      }
+    }
+
+    // in other cases like rest call, client operations, we get it from the current thread
+    currentUser = SecurityUtils.getSubject();
+
+    if (currentUser == null || currentUser.getPrincipal() == null) {
+      throw new GemFireSecurityException("Error: Anonymous User");
+    }
+
+    return currentUser;
+  }
+
+  /**
+   * @return return a shiro subject
+   */
+  @Override
+  public Subject login(final Properties credentials) {
+    if (credentials == null) {
+      throw new AuthenticationRequiredException("credentials are null");
+    }
+
+    // this makes sure it starts with a clean user object
+    ThreadContext.remove();
+
+    Subject currentUser = SecurityUtils.getSubject();
+    GeodeAuthenticationToken token = new GeodeAuthenticationToken(credentials);
+    try {
+      logger.debug("Logging in " + token.getPrincipal());
+      currentUser.login(token);
+    } catch (ShiroException e) {
+      logger.info(e.getMessage(), e);
+      throw new AuthenticationFailedException(
+          "Authentication error. Please check your credentials.", e);
+    }
+
+    return currentUser;
+  }
+
+  @Override
+  public void logout() {
+    Subject currentUser = getSubject();
+    try {
+      logger.debug("Logging out " + currentUser.getPrincipal());
+      currentUser.logout();
+    } catch (ShiroException e) {
+      logger.info(e.getMessage(), e);
+      throw new GemFireSecurityException(e.getMessage(), e);
+    }
+
+    // clean out Shiro's thread local content
+    ThreadContext.remove();
+  }
+
+  @Override
+  public Callable associateWith(final Callable callable) {
+    Subject currentUser = getSubject();
+    return currentUser.associateWith(callable);
+  }
+
+  /**
+   * Binds the passed-in subject to the executing thread. Usage:
+   *
+   * <pre>
+   * ThreadState state = null;
+   * try {
+   *   state = securityService.bindSubject(subject);
+   *   // do the rest of the work as this subject
+   * } finally {
+   *   if (state != null)
+   *     state.clear();
+   * }
+   * </pre>
+   */
+  @Override
+  public ThreadState bindSubject(final Subject subject) {
+    if (subject == null) {
+      throw new GemFireSecurityException("Error: Anonymous User");
+    }
+
+    ThreadState threadState = new SubjectThreadState(subject);
+    threadState.bind();
+    return threadState;
+  }
+
+  public void authorizeClusterManage() {
+    authorize(Resource.CLUSTER, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeClusterWrite() {
+    authorize(Resource.CLUSTER, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeClusterRead() {
+    authorize(Resource.CLUSTER, Operation.READ, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeDataManage() {
+    authorize(Resource.DATA, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeDataWrite() {
+    authorize(Resource.DATA, Operation.WRITE, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeDataRead() {
+    authorize(Resource.DATA, Operation.READ, Target.ALL, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeDiskManage() {
+    authorize(Resource.CLUSTER, Operation.MANAGE, Target.DISK, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeGatewayManage() {
+    authorize(Resource.CLUSTER, Operation.MANAGE, Target.GATEWAY, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeJarManage() {
+    authorize(Resource.CLUSTER, Operation.MANAGE, Target.JAR, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeQueryManage() {
+    authorize(Resource.CLUSTER, Operation.MANAGE, Target.QUERY, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeRegionManage(final String regionName) {
+    authorize(Resource.DATA, Operation.MANAGE, regionName, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeRegionManage(final String regionName, final String key) {
+    authorize(Resource.DATA, Operation.MANAGE, regionName, key);
+  }
+
+  @Override
+  public void authorizeRegionWrite(final String regionName) {
+    authorize(Resource.DATA, Operation.WRITE, regionName, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeRegionWrite(final String regionName, final String key) {
+    authorize(Resource.DATA, Operation.WRITE, regionName, key);
+  }
+
+  @Override
+  public void authorizeRegionRead(final String regionName) {
+    authorize(Resource.DATA, Operation.READ, regionName, ResourcePermission.ALL);
+  }
+
+  @Override
+  public void authorizeRegionRead(final String regionName, final String key) {
+    authorize(Resource.DATA, Operation.READ, regionName, key);
+  }
+
+  public void authorize(Resource resource, Operation operation, Target target, String key) {
+    authorize(resource, operation, target.getName(), key);
+  }
+
+  public void authorize(Resource resource, Operation operation, Target target) {
+    authorize(resource, operation, target, ResourcePermission.ALL);
+  }
+
+  public void authorize(Resource resource, Operation operation, String target, String key) {
+    authorize(new ResourcePermission(resource, operation, target, key));
+  }
+
+  @Override
+  public void authorize(final ResourcePermission context) {
+    if (context == null) {
+      return;
+    }
+    if (context.getResource() == Resource.NULL && context.getOperation() == Operation.NULL) {
+      return;
+    }
+
+    Subject currentUser = getSubject();
+    try {
+      currentUser.checkPermission(context);
+    } catch (ShiroException e) {
+      String msg = currentUser.getPrincipal() + " not authorized for " + context;
+      logger.info(msg);
+      throw new NotAuthorizedException(msg, e);
+    }
+  }
+
+  @Override
+  public void close() {
+    if (this.securityManager != null) {
+      securityManager.close();
+    }
+    if (this.postProcessor != null) {
+      this.postProcessor.close();
+    }
+
+    ThreadContext.remove();
+    SecurityUtils.setSecurityManager(null);
+  }
+
+  /**
+   * postProcess call already has this logic built in, you don't need to call this everytime you
+   * call postProcess. But if your postProcess is pretty involved with preparations and you need to
+   * bypass it entirely, call this first.
+   */
+  @Override
+  public boolean needPostProcess() {
+    return this.postProcessor != null;
+  }
+
+  @Override
+  public Object postProcess(final String regionPath, final Object key, final Object value,
+      final boolean valueIsSerialized) {
+    return postProcess(null, regionPath, key, value, valueIsSerialized);
+  }
+
+  @Override
+  public Object postProcess(Object principal, final String regionPath, final Object key,
+      final Object value, final boolean valueIsSerialized) {
+    if (!needPostProcess()) {
+      return value;
+    }
+
+    if (principal == null) {
+      principal = getSubject().getPrincipal();
+    }
+
+    String regionName = StringUtils.stripStart(regionPath, "/");
+    Object newValue;
+
+    // if the data is a byte array, but the data itself is supposed to be an object, we need to
+    // deserialize it before we pass it to the callback.
+    if (valueIsSerialized && value instanceof byte[]) {
+      try {
+        Object oldObj = EntryEventImpl.deserialize((byte[]) value);
+        Object newObj = this.postProcessor.processRegionValue(principal, regionName, key, oldObj);
+        newValue = BlobHelper.serializeToBlob(newObj);
+      } catch (IOException | SerializationException e) {
+        throw new GemFireIOException("Exception de/serializing entry value", e);
+      }
+    } else {
+      newValue = this.postProcessor.processRegionValue(principal, regionName, key, value);
+    }
+
+    return newValue;
+  }
+
+  @Override
+  public boolean isIntegratedSecurity() {
+    return true;
+  }
+
+  @Override
+  public boolean isClientSecurityRequired() {
+    return true;
+  }
+
+  @Override
+  public boolean isPeerSecurityRequired() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
index 4456253..c594bf9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
@@ -14,17 +14,20 @@
  */
 package org.apache.geode.internal.security;
 
-import org.apache.geode.security.PostProcessor;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.SecurityManager;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.shiro.subject.Subject;
 import org.apache.shiro.util.ThreadState;
 
-import java.util.Properties;
-import java.util.concurrent.Callable;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.PostProcessor;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.security.SecurityManager;
 
 /**
- * Legacy security service with ClientAuthenticator and/or PeerAuthenticator.
+ * implementing SecurityService when only legacy authenticators are specified
  */
 public class LegacySecurityService implements SecurityService {
 
@@ -32,164 +35,14 @@ public class LegacySecurityService implements SecurityService {
 
   private final boolean hasPeerAuthenticator;
 
-  LegacySecurityService(final String clientAuthenticator, final String peerAuthenticator) {
-    this.hasClientAuthenticator = clientAuthenticator != null;
-    this.hasPeerAuthenticator = peerAuthenticator != null;
-  }
-
-  @Override
-  public void initSecurity(final Properties securityProps) {
-    // nothing
-  }
-
-  @Override
-  public ThreadState bindSubject(final Subject subject) {
-    return null;
-  }
-
-  @Override
-  public Subject getSubject() {
-    return null;
-  }
-
-  @Override
-  public Subject login(final Properties credentials) {
-    return null;
-  }
-
-  @Override
-  public void logout() {
-    // nothing
-  }
-
-  @Override
-  public Callable associateWith(final Callable callable) {
-    return null;
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, String target, String key) {
-
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, ResourcePermission.Target target, String key) {
-
-  }
-
-  @Override
-  public void authorize(ResourcePermission.Resource resource,
-      ResourcePermission.Operation operation, ResourcePermission.Target target) {
-
-  }
-
-  @Override
-  public void authorizeClusterManage() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeClusterWrite() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeClusterRead() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataManage() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataWrite() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDataRead() {
-    // nothing
-  }
-
-  @Override
-  public void authorizeDiskManage() {
-
-  }
-
-  @Override
-  public void authorizeGatewayManage() {
-
-  }
-
-  @Override
-  public void authorizeJarManage() {
-
+  LegacySecurityService() {
+    hasClientAuthenticator = false;
+    hasPeerAuthenticator = false;
   }
 
-  @Override
-  public void authorizeQueryManage() {
-
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionManage(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionWrite(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName) {
-    // nothing
-  }
-
-  @Override
-  public void authorizeRegionRead(final String regionName, final String key) {
-    // nothing
-  }
-
-  @Override
-  public void authorize(final ResourcePermission context) {
-    // nothing
-  }
-
-  @Override
-  public void close() {
-    // nothing
-  }
-
-  @Override
-  public boolean needPostProcess() {
-    return false;
-  }
-
-  @Override
-  public Object postProcess(final String regionPath, final Object key, final Object value,
-      final boolean valueIsSerialized) {
-    return value;
-  }
-
-  @Override
-  public Object postProcess(final Object principal, final String regionPath, final Object key,
-      final Object value, final boolean valueIsSerialized) {
-    return value;
+  LegacySecurityService(final String clientAuthenticator, final String peerAuthenticator) {
+    this.hasClientAuthenticator = StringUtils.isNotBlank(clientAuthenticator);
+    this.hasPeerAuthenticator = StringUtils.isNotBlank(peerAuthenticator);
   }
 
   @Override
@@ -207,13 +60,4 @@ public class LegacySecurityService implements SecurityService {
     return this.hasPeerAuthenticator;
   }
 
-  @Override
-  public SecurityManager getSecurityManager() {
-    return null;
-  }
-
-  @Override
-  public PostProcessor getPostProcessor() {
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
index a4041e1..feea899 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
@@ -28,76 +28,97 @@ import java.util.Properties;
 import java.util.concurrent.Callable;
 
 public interface SecurityService {
+  default ThreadState bindSubject(Subject subject) {
+    return null;
+  }
 
-  void initSecurity(Properties securityProps);
+  default Subject getSubject() {
+    return null;
+  }
 
-  ThreadState bindSubject(Subject subject);
+  default Subject login(Properties credentials) {
+    return null;
+  }
 
-  Subject getSubject();
+  default void logout() {};
 
-  Subject login(Properties credentials);
+  default Callable associateWith(Callable callable) {
+    return callable;
+  }
 
-  void logout();
+  default void authorize(Resource resource, Operation operation, String target, String key) {};
 
-  Callable associateWith(Callable callable);
+  default void authorize(Resource resource, Operation operation, Target target, String key) {};
 
-  void authorize(Resource resource, Operation operation, String target, String key);
+  default void authorize(Resource resource, Operation operation, Target target) {};
 
-  void authorize(Resource resource, Operation operation, Target target, String key);
+  default void authorizeClusterManage() {};
 
-  void authorize(Resource resource, Operation operation, Target target);
+  default void authorizeClusterWrite() {};
 
-  void authorizeClusterManage();
+  default void authorizeClusterRead() {};
 
-  void authorizeClusterWrite();
+  default void authorizeDataManage() {};
 
-  void authorizeClusterRead();
+  default void authorizeDataWrite() {};
 
-  void authorizeDataManage();
+  default void authorizeDataRead() {};
 
-  void authorizeDataWrite();
+  default void authorizeDiskManage() {};
 
-  void authorizeDataRead();
+  default void authorizeGatewayManage() {};
 
-  void authorizeDiskManage();
+  default void authorizeJarManage() {};
 
-  void authorizeGatewayManage();
+  default void authorizeQueryManage() {};
 
-  void authorizeJarManage();
+  default void authorizeRegionManage(String regionName) {};
 
-  void authorizeQueryManage();
+  default void authorizeRegionManage(String regionName, String key) {};
 
-  void authorizeRegionManage(String regionName);
+  default void authorizeRegionWrite(String regionName) {};
 
-  void authorizeRegionManage(String regionName, String key);
+  default void authorizeRegionWrite(String regionName, String key) {};
 
-  void authorizeRegionWrite(String regionName);
+  default void authorizeRegionRead(String regionName) {};
 
-  void authorizeRegionWrite(String regionName, String key);
+  default void authorizeRegionRead(String regionName, String key) {};
 
-  void authorizeRegionRead(String regionName);
+  default void authorize(ResourcePermission context) {};
 
-  void authorizeRegionRead(String regionName, String key);
+  default void close() {};
 
-  void authorize(ResourcePermission context);
+  default boolean needPostProcess() {
+    return false;
+  };
 
-  void close();
+  default Object postProcess(String regionPath, Object key, Object value,
+      boolean valueIsSerialized) {
+    return value;
+  }
 
-  boolean needPostProcess();
+  default Object postProcess(Object principal, String regionPath, Object key, Object value,
+      boolean valueIsSerialized) {
+    return value;
+  }
 
-  Object postProcess(String regionPath, Object key, Object value, boolean valueIsSerialized);
+  default boolean isClientSecurityRequired() {
+    return false;
+  }
 
-  Object postProcess(Object principal, String regionPath, Object key, Object value,
-      boolean valueIsSerialized);
+  default boolean isIntegratedSecurity() {
+    return false;
+  }
 
-  boolean isClientSecurityRequired();
+  default boolean isPeerSecurityRequired() {
+    return false;
+  }
 
-  boolean isIntegratedSecurity();
-
-  boolean isPeerSecurityRequired();
-
-  SecurityManager getSecurityManager();
-
-  PostProcessor getPostProcessor();
+  default SecurityManager getSecurityManager() {
+    return null;
+  }
 
+  default PostProcessor getPostProcessor() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
index 02f34f1..2e0ad95 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
@@ -14,25 +14,22 @@
  */
 package org.apache.geode.internal.security;
 
-import static org.apache.commons.lang.StringUtils.isNotBlank;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_PEER_AUTHENTICATOR;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_SHIRO_INIT;
 
-import org.apache.geode.distributed.internal.DistributionConfig;
+import java.util.Properties;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.UnavailableSecurityManagerException;
+
 import org.apache.geode.internal.cache.CacheConfig;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.security.shiro.ConfigInitializer;
-import org.apache.geode.internal.security.shiro.RealmInitializer;
+import org.apache.geode.internal.security.shiro.SecurityManagerProvider;
 import org.apache.geode.security.PostProcessor;
 import org.apache.geode.security.SecurityManager;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.UnavailableSecurityManagerException;
-
-import java.util.Properties;
 
 public class SecurityServiceFactory {
 
@@ -40,119 +37,65 @@ public class SecurityServiceFactory {
     // do not instantiate
   }
 
-  public static SecurityService create(CacheConfig cacheConfig,
-      DistributionConfig distributionConfig) {
-    Properties securityConfig = getSecurityConfig(distributionConfig);
-    SecurityManager securityManager =
-        getSecurityManager(getSecurityManagerFromConfig(cacheConfig), securityConfig);
-    PostProcessor postProcessor =
-        getPostProcessor(getPostProcessorFromConfig(cacheConfig), securityConfig);
-    return create(distributionConfig, securityManager, postProcessor);
-  }
-
-  /**
-   * Creates and initializes SecurityService. Initialization will invoke init on both
-   * SecurityManager and PostProcessor if they are specified.
-   */
-  public static SecurityService create(DistributionConfig distributionConfig,
-      SecurityManager securityManager, PostProcessor postProcessor) {
-    Properties securityConfig = getSecurityConfig(distributionConfig);
-
-    securityManager = getSecurityManager(securityManager, securityConfig);
-    postProcessor = getPostProcessor(postProcessor, securityConfig);
-
-    SecurityService securityService = create(securityConfig, securityManager, postProcessor);
-    initialize(securityService, distributionConfig);
-    return securityService;
-  }
-
   public static SecurityService create() {
-    return new DisabledSecurityService();
+    return new LegacySecurityService();
   }
 
-  public static SecurityService create(Properties securityConfig, SecurityManager securityManager,
-      PostProcessor postProcessor) {
-    SecurityServiceType type = determineType(securityConfig, securityManager, postProcessor);
-    switch (type) {
-      case CUSTOM:
-        String shiroConfig = getProperty(securityConfig, SECURITY_SHIRO_INIT);
-        if (isNotBlank(shiroConfig)) {
-          new ConfigInitializer().initialize(shiroConfig);
-        }
-        return new CustomSecurityService(postProcessor);
-      case ENABLED:
-        return new EnabledSecurityService(securityManager, postProcessor, new RealmInitializer());
-      case LEGACY:
-        String clientAuthenticator = getProperty(securityConfig, SECURITY_CLIENT_AUTHENTICATOR);
-        String peerAuthenticator = getProperty(securityConfig, SECURITY_PEER_AUTHENTICATOR);
-        return new LegacySecurityService(clientAuthenticator, peerAuthenticator);
-      default:
-        return new DisabledSecurityService();
-    }
+  public static SecurityService create(Properties securityProps) {
+    return create(securityProps, null, null);
   }
 
-  public static SecurityService findSecurityService() {
-    InternalCache cache = GemFireCacheImpl.getInstance();
-    if (cache != null) {
-      return cache.getSecurityService();
+  public static SecurityService create(Properties securityProps, CacheConfig cacheConfig) {
+    if (cacheConfig == null) {
+      return create(securityProps, null, null);
     }
-    return SecurityServiceFactory.create();
-  }
 
-  static SecurityServiceType determineType(Properties securityConfig,
-      SecurityManager securityManager, PostProcessor postProcessor) {
-    boolean hasShiroConfig = hasProperty(securityConfig, SECURITY_SHIRO_INIT);
-    if (hasShiroConfig) {
-      return SecurityServiceType.CUSTOM;
-    }
+    return create(securityProps, cacheConfig.getSecurityManager(), cacheConfig.getPostProcessor());
+  }
 
-    boolean hasSecurityManager =
-        securityManager != null || hasProperty(securityConfig, SECURITY_MANAGER);
-    if (hasSecurityManager) {
-      return SecurityServiceType.ENABLED;
+  public static SecurityService create(Properties securityProps,
+      SecurityManager preferredSecurityManager, PostProcessor preferredPostProcessor) {
+    if (securityProps == null) {
+      // avoid NPE, and we can stil use preferredSecurityManager to create the service
+      securityProps = new Properties();
     }
 
-    boolean hasClientAuthenticator = hasProperty(securityConfig, SECURITY_CLIENT_AUTHENTICATOR);
-    boolean hasPeerAuthenticator = hasProperty(securityConfig, SECURITY_PEER_AUTHENTICATOR);
-    if (hasClientAuthenticator || hasPeerAuthenticator) {
-      return SecurityServiceType.LEGACY;
-    }
+    String shiroConfig = securityProps.getProperty(SECURITY_SHIRO_INIT);
+    SecurityManager securityManager = CallbackInstantiator.getSecurityManager(securityProps);
+    PostProcessor postProcessor = CallbackInstantiator.getPostProcessor(securityProps);
 
-    boolean isShiroInUse = isShiroInUse();
-    if (isShiroInUse) {
-      return SecurityServiceType.CUSTOM;
+    // cacheConfig's securityManager/postprocessor takes precedence over those defined in
+    // securityProps
+    if (preferredSecurityManager != null) {
+      // cacheConfig's security manager will override property's shiro.ini settings
+      shiroConfig = null;
+      securityManager = preferredSecurityManager;
     }
-
-    return SecurityServiceType.DISABLED;
-  }
-
-  static SecurityManager getSecurityManager(SecurityManager securityManager,
-      Properties securityConfig) {
-    if (securityManager != null) {
-      return securityManager;
+    if (preferredPostProcessor != null) {
+      postProcessor = preferredPostProcessor;
     }
 
-    String securityManagerConfig = getProperty(securityConfig, SECURITY_MANAGER);
-    if (isNotBlank(securityManagerConfig)) {
-      securityManager = CallbackInstantiator.getObjectOfTypeFromClassName(securityManagerConfig,
-          SecurityManager.class);
+    if (StringUtils.isNotBlank(shiroConfig)) {
+      return new IntegratedSecurityService(new SecurityManagerProvider(shiroConfig), postProcessor);
+    } else if (securityManager != null) {
+      return new IntegratedSecurityService(new SecurityManagerProvider(securityManager),
+          postProcessor);
+    } else if (isShiroInUse()) {
+      return new IntegratedSecurityService(new SecurityManagerProvider(), postProcessor);
     }
 
-    return securityManager;
+    // if not return legacy security service
+    String clientAuthenticatorConfig = securityProps.getProperty(SECURITY_CLIENT_AUTHENTICATOR);
+    String peerAuthenticatorConfig = securityProps.getProperty(SECURITY_PEER_AUTHENTICATOR);
+    return new LegacySecurityService(clientAuthenticatorConfig, peerAuthenticatorConfig);
   }
 
-  static PostProcessor getPostProcessor(PostProcessor postProcessor, Properties securityConfig) {
-    if (postProcessor != null) {
-      return postProcessor;
-    }
-
-    String postProcessorConfig = getProperty(securityConfig, SECURITY_POST_PROCESSOR);
-    if (isNotBlank(postProcessorConfig)) {
-      postProcessor = CallbackInstantiator.getObjectOfTypeFromClassName(postProcessorConfig,
-          PostProcessor.class);
+  public static SecurityService findSecurityService() {
+    InternalCache cache = GemFireCacheImpl.getInstance();
+    if (cache != null) {
+      return cache.getSecurityService();
     }
-
-    return postProcessor;
+    return SecurityServiceFactory.create();
   }
 
   private static boolean isShiroInUse() {
@@ -163,42 +106,5 @@ public class SecurityServiceFactory {
     }
   }
 
-  private static Properties getSecurityConfig(DistributionConfig distributionConfig) {
-    if (distributionConfig == null) {
-      return new Properties();
-    }
-    return distributionConfig.getSecurityProps();
-  }
 
-  private static SecurityManager getSecurityManagerFromConfig(CacheConfig cacheConfig) {
-    if (cacheConfig == null) {
-      return null;
-    }
-    return cacheConfig.getSecurityManager();
-  }
-
-  private static PostProcessor getPostProcessorFromConfig(CacheConfig cacheConfig) {
-    if (cacheConfig == null) {
-      return null;
-    }
-    return cacheConfig.getPostProcessor();
-  }
-
-  private static boolean hasProperty(Properties securityConfig, String key) {
-    return securityConfig != null && getProperty(securityConfig, key) != null;
-  }
-
-  private static String getProperty(Properties securityConfig, String key) {
-    if (securityConfig == null) {
-      return null;
-    }
-    return securityConfig.getProperty(key);
-  }
-
-  private static void initialize(SecurityService securityService,
-      DistributionConfig distributionConfig) {
-    if (securityService != null && distributionConfig != null) {
-      securityService.initSecurity(distributionConfig.getSecurityProps());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceType.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceType.java b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceType.java
deleted file mode 100644
index 8ae76d2..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceType.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security;
-
-public enum SecurityServiceType {
-  /** Integrated Security is Enabled */
-  ENABLED,
-  /** Security is Disabled */
-  DISABLED,
-  /** Legacy Security is Enabled */
-  LEGACY,
-  /** Shiro Config is specified */
-  CUSTOM,
-  /** Shiro is already configured with SecurityManager */
-  EXTERNAL
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/shiro/ConfigInitializer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/ConfigInitializer.java b/geode-core/src/main/java/org/apache/geode/internal/security/shiro/ConfigInitializer.java
deleted file mode 100644
index 60f014b..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/ConfigInitializer.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security.shiro;
-
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.config.Ini.Section;
-import org.apache.shiro.config.IniSecurityManagerFactory;
-import org.apache.shiro.mgt.SecurityManager;
-
-public class ConfigInitializer {
-
-  public ConfigInitializer() {
-    // nothing
-  }
-
-  public void initialize(String shiroConfig) {
-    IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:" + shiroConfig);
-
-    // we will need to make sure that shiro uses a case sensitive permission resolver
-    Section main = factory.getIni().addSection("main");
-    main.put("geodePermissionResolver", GeodePermissionResolver.class.getName());
-    if (!main.containsKey("iniRealm.permissionResolver")) {
-      main.put("iniRealm.permissionResolver", "$geodePermissionResolver");
-    }
-
-    // logs "Users or Roles are already populated. Configured Ini instance will be ignored."
-    SecurityManager securityManager = factory.getInstance();
-    SecurityUtils.setSecurityManager(securityManager);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/shiro/CustomAuthRealm.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/CustomAuthRealm.java b/geode-core/src/main/java/org/apache/geode/internal/security/shiro/CustomAuthRealm.java
index 51449fd..0e5029b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/CustomAuthRealm.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/shiro/CustomAuthRealm.java
@@ -14,10 +14,6 @@
  */
 package org.apache.geode.internal.security.shiro;
 
-import java.io.Serializable;
-import java.util.Properties;
-
-import org.apache.geode.internal.security.CallbackInstantiator;
 import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.SecurityManager;
 import org.apache.shiro.authc.AuthenticationException;
@@ -29,6 +25,9 @@ import org.apache.shiro.authz.Permission;
 import org.apache.shiro.realm.AuthorizingRealm;
 import org.apache.shiro.subject.PrincipalCollection;
 
+import java.io.Serializable;
+import java.util.Properties;
+
 public class CustomAuthRealm extends AuthorizingRealm {
 
   private static final String REALM_NAME = "CUSTOMAUTHREALM";
@@ -46,18 +45,6 @@ public class CustomAuthRealm extends AuthorizingRealm {
     setAuthenticationTokenClass(GeodeAuthenticationToken.class);
   }
 
-  /**
-   * SecurityManager will be constructed and initialized with the provided security properties.
-   *
-   * @param authenticatorFactory name of the SecurityManager implementation to construct
-   * @param securityProperties the security properties to initialize SecurityManager with
-   */
-  public CustomAuthRealm(String authenticatorFactory, Properties securityProperties) {
-    this.securityManager = CallbackInstantiator.getObjectOfTypeFromClassName(authenticatorFactory,
-        SecurityManager.class);
-    this.securityManager.init(securityProperties);
-  }
-
   @Override
   protected AuthenticationInfo doGetAuthenticationInfo(AuthenticationToken token)
       throws AuthenticationException {

http://git-wip-us.apache.org/repos/asf/geode/blob/5546a873/geode-core/src/main/java/org/apache/geode/internal/security/shiro/RealmInitializer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/RealmInitializer.java b/geode-core/src/main/java/org/apache/geode/internal/security/shiro/RealmInitializer.java
deleted file mode 100644
index 978c4dd..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/security/shiro/RealmInitializer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.security.shiro;
-
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.security.SecurityManager;
-import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.mgt.DefaultSecurityManager;
-import org.apache.shiro.realm.Realm;
-import org.apache.shiro.session.mgt.DefaultSessionManager;
-import org.apache.shiro.session.mgt.SessionManager;
-
-public class RealmInitializer {
-  private static Logger logger = LogService.getLogger(LogService.SECURITY_LOGGER_NAME);
-
-  public RealmInitializer() {
-    // nothing
-  }
-
-  public void initialize(final SecurityManager securityManager) {
-    Realm realm = new CustomAuthRealm(securityManager);
-    DefaultSecurityManager shiroManager = new DefaultSecurityManager(realm);
-    SecurityUtils.setSecurityManager(shiroManager);
-    increaseShiroGlobalSessionTimeout(shiroManager);
-  }
-
-  private void increaseShiroGlobalSessionTimeout(final DefaultSecurityManager shiroManager) {
-    SessionManager sessionManager = shiroManager.getSessionManager();
-    if (DefaultSessionManager.class.isInstance(sessionManager)) {
-      DefaultSessionManager defaultSessionManager = (DefaultSessionManager) sessionManager;
-      defaultSessionManager.setGlobalSessionTimeout(Long.MAX_VALUE);
-      long value = defaultSessionManager.getGlobalSessionTimeout();
-      if (value != Long.MAX_VALUE) {
-        logger.error("Unable to set Shiro Global Session Timeout. Current value is '{}'.", value);
-      }
-    } else {
-      logger.error("Unable to set Shiro Global Session Timeout. Current SessionManager is '{}'.",
-          sessionManager == null ? "null" : sessionManager.getClass());
-    }
-  }
-}