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

[geode] 01/08: Moved NoOpAuthenticator.java and NoOpAuthorizer.java out of core

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

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

commit 4ed66660c2dc6a3e9f9d6f9451faf6e1a85f1b02
Author: kohlmu-pivotal <uk...@pivotal.io>
AuthorDate: Fri Sep 8 15:48:07 2017 -0700

    Moved NoOpAuthenticator.java and NoOpAuthorizer.java out of core
---
 .../distributed/internal/tcpserver/TcpServer.java  | 45 ++++++++--------------
 .../sockets/GenericProtocolServerConnection.java   | 24 ++++++------
 .../tier/sockets/MessageExecutionContext.java      |  9 ++---
 .../tier/sockets/ServerConnectionFactory.java      | 12 +++---
 .../apache/geode/security/server/Authorizer.java   |  4 +-
 .../org.apache.geode.security.server.Authenticator |  2 +-
 .../protocol/protobuf/ProtobufOpsProcessor.java    | 22 +++++++++--
 .../protobuf/ProtobufSimpleAuthenticator.java      | 44 +++++++++++----------
 .../geode/security/server/NoOpAuthenticator.java   |  0
 .../geode/security/server/NoOpAuthorizer.java      |  0
 .../org.apache.geode.security.server.Authenticator |  3 +-
 .../GenericProtocolServerConnectionTest.java       |  0
 .../RoundTripCacheConnectionJUnitTest.java         |  7 +---
 13 files changed, 86 insertions(+), 86 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
index a721589..6ec265a 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpServer.java
@@ -68,27 +68,20 @@ import org.apache.geode.internal.security.SecurableCommunicationChannel;
 
 /**
  * TCP server which listens on a port and delegates requests to a request handler. The server uses
- * expects messages containing a global version number, followed by a DataSerializable object
- * <p>
+ * expects messages containing a global version number, followed by a DataSerializable object <p>
  * This code was factored out of GossipServer.java to allow multiple handlers to share the same
  * gossip server port.
- * 
  * @since GemFire 5.7
  */
 public class TcpServer {
 
   /**
-   * The version of the tcp server protocol
-   * <p>
-   * This should be incremented if the gossip message structures change
-   * <p>
-   * 0 - special indicator of a non-gossip message from a client<br>
-   * 1000 - gemfire 5.5 - using java serialization<br>
-   * 1001 - 5.7 - using DataSerializable and supporting server locator messages.<br>
-   * 1002 - 7.1 - sending GemFire version along with GOSSIP_VERSION in each request.
-   * <p>
-   * with the addition of support for all old versions of clients you can no longer change this
-   * version number
+   * The version of the tcp server protocol <p> This should be incremented if the gossip message
+   * structures change <p> 0 - special indicator of a non-gossip message from a client<br> 1000 -
+   * gemfire 5.5 - using java serialization<br> 1001 - 5.7 - using DataSerializable and supporting
+   * server locator messages.<br> 1002 - 7.1 - sending GemFire version along with GOSSIP_VERSION in
+   * each request. <p> with the addition of support for all old versions of clients you can no
+   * longer change this version number
    */
   public final static int GOSSIPVERSION = 1002;
   public final static int NON_GOSSIP_REQUEST_VERSION = 0;
@@ -97,12 +90,7 @@ public class TcpServer {
   // GossipServer.
   public final static int OLDGOSSIPVERSION = 1001;
 
-  private static/* GemStoneAddition */ final Map GOSSIP_TO_GEMFIRE_VERSION_MAP = new HashMap();
-
-  /**
-   * For the new client-server protocol, which ignores the usual handshake mechanism.
-   */
-  public static final byte PROTOBUF_CLIENT_SERVER_PROTOCOL = (byte) 110;
+  private static final Map GOSSIP_TO_GEMFIRE_VERSION_MAP = new HashMap();
 
   // For test purpose only
   public static boolean isTesting = false;
@@ -117,7 +105,7 @@ public class TcpServer {
 
   private static final Logger log = LogService.getLogger();
 
-  protected/* GemStoneAddition */ final/* GemStoneAddition */ static int READ_TIMEOUT =
+  protected final static int READ_TIMEOUT =
       Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "TcpServer.READ_TIMEOUT", 60 * 1000);
   // This is for backwards compatibility. The p2p.backlog flag used to be the only way to configure
   // the locator backlog.
@@ -156,9 +144,9 @@ public class TcpServer {
   }
 
   public TcpServer(int port, InetAddress bind_address, Properties sslConfig,
-      DistributionConfigImpl cfg, TcpHandler handler, PoolStatHelper poolHelper,
-      ThreadGroup threadGroup, String threadName, InternalLocator internalLocator,
-      ClientProtocolMessageHandler messageHandler) {
+                   DistributionConfigImpl cfg, TcpHandler handler, PoolStatHelper poolHelper,
+                   ThreadGroup threadGroup, String threadName, InternalLocator internalLocator,
+                   ClientProtocolMessageHandler messageHandler) {
     this.port = port;
     this.bind_address = bind_address;
     this.handler = handler;
@@ -191,7 +179,7 @@ public class TcpServer {
   }
 
   private static PooledExecutorWithDMStats createExecutor(PoolStatHelper poolHelper,
-      final ThreadGroup threadGroup) {
+                                                          final ThreadGroup threadGroup) {
     ThreadFactory factory = new ThreadFactory() {
       private final AtomicInteger threadNum = new AtomicInteger();
 
@@ -208,7 +196,7 @@ public class TcpServer {
   }
 
   public void restarting(InternalDistributedSystem ds, InternalCache cache,
-      ClusterConfigurationService sharedConfig) throws IOException {
+                         ClusterConfigurationService sharedConfig) throws IOException {
     this.shuttingDown = false;
     this.handler.restarting(ds, cache, sharedConfig);
     startServerThread();
@@ -278,7 +266,6 @@ public class TcpServer {
   /**
    * Returns the value of the bound port. If the server was initialized with a port of 0 indicating
    * that any ephemeral port should be used, this method will return the actual bound port.
-   * 
    * @return the locator's tcp/ip port. This will be zero if the locator hasn't been started.
    */
   public int getPort() {
@@ -372,7 +359,8 @@ public class TcpServer {
 
         short versionOrdinal;
         if (gossipVersion == NON_GOSSIP_REQUEST_VERSION) {
-          if (input.readUnsignedByte() == PROTOBUF_CLIENT_SERVER_PROTOCOL
+          if (input.readUnsignedByte() == CommunicationMode.ProtobufClientServerProtocol
+              .getModeNumber()
               && Boolean.getBoolean("geode.feature-protobuf-protocol")) {
             messageHandler.receiveMessage(input, socket.getOutputStream(),
                 new MessageExecutionContext(internalLocator));
@@ -538,7 +526,6 @@ public class TcpServer {
 
   /**
    * Returns GossipVersion for older Gemfire versions.
-   * 
    * @return gossip version
    */
   public static int getGossipVersionForOrdinal(short ordinal) {
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 6c81028..cf94af9 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
@@ -15,6 +15,7 @@
 
 package org.apache.geode.internal.cache.tier.sockets;
 
+import org.apache.geode.cache.Cache;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.Acceptor;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
@@ -41,11 +42,15 @@ public class GenericProtocolServerConnection extends ServerConnection {
    * Creates a new <code>GenericProtocolServerConnection</code> that processes messages received
    * from an edge client over a given <code>Socket</code>.
    */
-  public GenericProtocolServerConnection(Socket s, InternalCache c, CachedRegionHelper helper,
-      CacheServerStats stats, int hsTimeout, int socketBufferSize, String communicationModeStr,
-      byte communicationMode, Acceptor acceptor, ClientProtocolMessageHandler newClientProtocol,
-      SecurityService securityService, Authenticator authenticator) {
-    super(s, c, helper, stats, hsTimeout, socketBufferSize, communicationModeStr, communicationMode,
+  GenericProtocolServerConnection(Socket socket, InternalCache cache,
+                                  CachedRegionHelper cachedRegionHelper,
+                                  CacheServerStats cacheServerStats, int hsTimeout,
+                                  int socketBufferSize, String communicationModeStr,
+                                  byte communicationMode, Acceptor acceptor,
+                                  ClientProtocolMessageHandler newClientProtocol,
+                                  SecurityService securityService, Authenticator authenticator) {
+    super(socket, cache, cachedRegionHelper, cacheServerStats, hsTimeout, socketBufferSize,
+        communicationModeStr, communicationMode,
         acceptor, securityService);
     securityManager = securityService.getSecurityManager();
     this.messageHandler = newClientProtocol;
@@ -59,12 +64,9 @@ public class GenericProtocolServerConnection extends ServerConnection {
       InputStream inputStream = socket.getInputStream();
       OutputStream outputStream = socket.getOutputStream();
 
-      if (!authenticator.isAuthenticated()) {
-        authenticator.authenticate(inputStream, outputStream, securityManager);
-      } else {
-        messageHandler.receiveMessage(inputStream, outputStream,
-            new MessageExecutionContext(this.getCache(), authenticator.getAuthorizer()));
-      }
+      authenticator.authenticate(inputStream, outputStream, securityManager);
+      messageHandler.receiveMessage(inputStream, outputStream,
+          new MessageExecutionContext(this.getCache(), authenticator.getAuthorizer()));
     } catch (EOFException e) {
       this.setFlagProcessMessagesAsFalse();
       setClientDisconnectedException(e);
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 1130ce7..d68a165 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,9 +18,7 @@ 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.Locator;
-import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.exception.InvalidExecutionContextException;
-import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.security.server.Authorizer;
 
 @Experimental
@@ -34,14 +32,13 @@ public class MessageExecutionContext {
     this.authorizer = streamAuthorizer;
   }
 
-  public MessageExecutionContext(InternalLocator locator) {
+
+  public MessageExecutionContext(Locator locator) {
     this.locator = locator;
-    // set a no-op authorizer until such time as locators implement authentication
-    // and authorization checks
-    this.authorizer = new NoOpAuthorizer();
   }
 
   /**
+
    * Returns the cache associated with this execution
    * <p>
    *
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 f0348c3..722cd34 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
@@ -36,8 +36,6 @@ public class ServerConnectionFactory {
   private ClientProtocolMessageHandler protocolHandler;
   private Map<String, Class<? extends Authenticator>> authenticators = null;
 
-  public ServerConnectionFactory() {}
-
   private synchronized void initializeAuthenticatorsMap() {
     if (authenticators != null) {
       return;
@@ -85,23 +83,23 @@ public class ServerConnectionFactory {
     return protocolHandler;
   }
 
-  public ServerConnection makeServerConnection(Socket s, InternalCache c, CachedRegionHelper helper,
-      CacheServerStats stats, int hsTimeout, int socketBufferSize, String communicationModeStr,
+  public ServerConnection makeServerConnection(Socket socket, InternalCache cache, CachedRegionHelper cachedRegionHelper,
+      CacheServerStats cacheServerStats, int hsTimeout, int socketBufferSize, String communicationModeStr,
       byte communicationMode, Acceptor acceptor, SecurityService securityService)
       throws IOException {
-    if (communicationMode == ProtobufClientServerProtocol.getModeNumber()) {
+    if (ProtobufClientServerProtocol.getModeNumber() == communicationMode) {
       if (!Boolean.getBoolean("geode.feature-protobuf-protocol")) {
         throw new IOException("Server received unknown communication mode: " + communicationMode);
       } else {
         String authenticationMode =
             System.getProperty("geode.protocol-authentication-mode", "NOOP");
 
-        return new GenericProtocolServerConnection(s, c, helper, stats, hsTimeout, socketBufferSize,
+        return new GenericProtocolServerConnection(socket, cache, cachedRegionHelper, cacheServerStats, hsTimeout, socketBufferSize,
             communicationModeStr, communicationMode, acceptor, getClientProtocolMessageHandler(),
             securityService, findStreamAuthenticator(authenticationMode));
       }
     } else {
-      return new LegacyServerConnection(s, c, helper, stats, hsTimeout, socketBufferSize,
+      return new LegacyServerConnection(socket, cache, cachedRegionHelper, cacheServerStats, hsTimeout, socketBufferSize,
           communicationModeStr, communicationMode, acceptor, securityService);
     }
   }
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
index fea2198..e3f5738 100644
--- 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
@@ -17,5 +17,7 @@ package org.apache.geode.security.server;
 import org.apache.geode.security.ResourcePermission;
 
 public interface Authorizer {
-  boolean authorize(ResourcePermission permissionRequested);
+  default boolean authorize(ResourcePermission permissionRequested) {
+    return true;
+  }
 }
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
index 4f34d2a..8b13789 100644
--- 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
@@ -1 +1 @@
-org.apache.geode.security.server.NoOpAuthenticator
+
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 7dee26b..d3ac9d7 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
@@ -20,6 +20,8 @@ import org.apache.geode.internal.exception.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.server.Authorizer;
+import org.apache.geode.security.server.NoOpAuthorizer;
 import org.apache.geode.serialization.SerializationService;
 
 /**
@@ -33,22 +35,23 @@ public class ProtobufOpsProcessor {
   private final SerializationService serializationService;
 
   public ProtobufOpsProcessor(SerializationService serializationService,
-      OperationContextRegistry operationContextRegistry) {
+                              OperationContextRegistry operationContextRegistry) {
     this.serializationService = serializationService;
     this.operationContextRegistry = operationContextRegistry;
   }
 
   public ClientProtocol.Response process(ClientProtocol.Request request,
-      MessageExecutionContext context) {
+                                         MessageExecutionContext context) {
     ClientProtocol.Request.RequestAPICase requestType = request.getRequestAPICase();
     OperationContext operationContext = operationContextRegistry.getOperationContext(requestType);
     ClientProtocol.Response.Builder builder;
     Result result;
+    Authorizer authorizer = findAuthorizer(context);
     try {
-      if (context.getAuthorizer().authorize(operationContext.getAccessPermissionRequired())) {
+      if (authorizer.authorize(operationContext.getAccessPermissionRequired())) {
         result = operationContext.getOperationHandler().process(serializationService,
             operationContext.getFromRequest().apply(request), context);
-      } else {
+      } else{
         result = Failure.of(ProtobufResponseUtilities.makeErrorResponse(
             ProtocolErrorCode.AUTHORIZATION_FAILED.codeValue,
             "User isn't authorized for this operation."));
@@ -63,4 +66,15 @@ public class ProtobufOpsProcessor {
         operationContext.getToErrorResponse());
     return builder.build();
   }
+
+  private Authorizer findAuthorizer(MessageExecutionContext context) {
+    Authorizer authorizer = context.getAuthorizer();
+    if(authorizer != null)
+    {
+      return authorizer;
+    }else{
+      //TODO Horrible HACK because we need to look this up correctly.
+      return new NoOpAuthorizer();
+    }
+  }
 }
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 5fadadd..47b7fac 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
@@ -30,38 +30,42 @@ import java.util.Properties;
 
 public class ProtobufSimpleAuthenticator implements Authenticator {
   private ProtobufSimpleAuthorizer authorizer = null;
+  private boolean authenticationSuccessfull = false;
 
   @Override
   public void authenticate(InputStream inputStream, OutputStream outputStream,
-      SecurityManager securityManager) throws IOException {
-    AuthenticationAPI.SimpleAuthenticationRequest authenticationRequest =
-        AuthenticationAPI.SimpleAuthenticationRequest.parseDelimitedFrom(inputStream);
-    if (authenticationRequest == null) {
-      throw new EOFException();
-    }
+                           SecurityManager securityManager) throws IOException {
+    if (!authenticationSuccessfull) {
+      AuthenticationAPI.SimpleAuthenticationRequest authenticationRequest =
+          AuthenticationAPI.SimpleAuthenticationRequest.parseDelimitedFrom(inputStream);
+      if (authenticationRequest == null) {
+        throw new EOFException();
+      }
 
-    Properties properties = new Properties();
-    properties.setProperty(ResourceConstants.USER_NAME, authenticationRequest.getUsername());
-    properties.setProperty(ResourceConstants.PASSWORD, authenticationRequest.getPassword());
+      Properties properties = new Properties();
+      properties.setProperty(ResourceConstants.USER_NAME, authenticationRequest.getUsername());
+      properties.setProperty(ResourceConstants.PASSWORD, authenticationRequest.getPassword());
 
-    authorizer = null; // authenticating a new user clears current authorizer
-    try {
-      Object principal = securityManager.authenticate(properties);
-      if (principal != null) {
-        authorizer = new ProtobufSimpleAuthorizer(principal, securityManager);
+      try {
+        Object principal = securityManager.authenticate(properties);
+        if (principal != null) {
+          authenticationSuccessfull = true;
+          authorizer = new ProtobufSimpleAuthorizer(principal, securityManager);
+        }
+      } catch (AuthenticationFailedException e) {
+        authorizer = null;
       }
-    } catch (AuthenticationFailedException e) {
-      authorizer = null;
+
+      AuthenticationAPI.SimpleAuthenticationResponse.newBuilder()
+          .setAuthenticated(isAuthenticated())
+          .build().writeDelimitedTo(outputStream);
     }
 
-    AuthenticationAPI.SimpleAuthenticationResponse.newBuilder().setAuthenticated(isAuthenticated())
-        .build().writeDelimitedTo(outputStream);
   }
 
   @Override
   public boolean isAuthenticated() {
-    // note: an authorizer is only created if the user has been authenticated
-    return authorizer != null;
+    return authenticationSuccessfull;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java b/geode-protobuf/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
rename to geode-protobuf/src/main/java/org/apache/geode/security/server/NoOpAuthenticator.java
diff --git a/geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthorizer.java b/geode-protobuf/src/main/java/org/apache/geode/security/server/NoOpAuthorizer.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/security/server/NoOpAuthorizer.java
rename to geode-protobuf/src/main/java/org/apache/geode/security/server/NoOpAuthorizer.java
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
index 45e4eea..e13c994 100644
--- 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
@@ -1 +1,2 @@
-org.apache.geode.protocol.protobuf.ProtobufSimpleAuthenticator
\ No newline at end of file
+org.apache.geode.protocol.protobuf.ProtobufSimpleAuthenticator
+org.apache.geode.security.server.NoOpAuthenticator
\ No newline at end of file
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
similarity index 100%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
rename to geode-protobuf/src/test/java/org/apache/geode/internal/cache/tier/sockets/GenericProtocolServerConnectionTest.java
diff --git a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
index cc15e4f..b38771b 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/protocol/RoundTripCacheConnectionJUnitTest.java
@@ -292,12 +292,7 @@ public class RoundTripCacheConnectionJUnitTest {
 
   @Test
   public void testNewProtocolRespectsMaxConnectionLimit() throws IOException, InterruptedException {
-    cache.getDistributedSystem().disconnect();
-
-    CacheFactory cacheFactory = new CacheFactory();
-    cacheFactory.set(ConfigurationProperties.LOCATORS, "");
-    cacheFactory.set(ConfigurationProperties.MCAST_PORT, "0");
-    cache = cacheFactory.create();
+    cache.getCacheServers().get(0).stop();
 
     CacheServer cacheServer = cache.addCacheServer();
     final int cacheServerPort = AvailablePortHelper.getRandomAvailableTCPPort();

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