You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/05/31 17:01:07 UTC

[02/32] geode git commit: GEODE-2970: clearing LogWriterAppender when shutting down locator.

GEODE-2970: clearing LogWriterAppender when shutting down locator.

* Do not bury the NPE in AlterRuntimeConfigFunction
* destroy the LogWriterAppender when removing the locator
* added test


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

Branch: refs/heads/feature/GEODE-1279
Commit: ca12f781c14409ee87873f604be64d98952c0a9a
Parents: f9099df
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon May 22 08:47:37 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed May 24 08:42:24 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/distributed/Locator.java   |  12 +-
 .../geode/distributed/LocatorLauncher.java      |   2 +-
 .../distributed/internal/InternalLocator.java   |  35 +--
 .../geode/internal/DistributionLocator.java     |   2 +-
 .../internal/cli/commands/ConfigCommands.java   | 245 +++++++++----------
 .../functions/AlterRuntimeConfigFunction.java   |  13 +-
 .../InternalLocatorIntegrationTest.java         |  79 ++++++
 .../cli/commands/ConfigCommandsDUnitTest.java   |  14 +-
 .../dunit/rules/LocatorServerStartupRule.java   |   5 +-
 .../apache/geode/test/dunit/rules/MemberVM.java |  11 +-
 10 files changed, 231 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/Locator.java b/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
index 87cd243..645b261 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/Locator.java
@@ -14,6 +14,10 @@
  */
 package org.apache.geode.distributed;
 
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.net.SocketCreator;
+
 import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -21,10 +25,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.internal.net.SocketCreator;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-
 /**
  * Represents a distribution locator server that provides discovery information to members and
  * clients of a GemFire distributed system. In most GemFire distributed cache architectures,
@@ -250,8 +250,8 @@ public abstract class Locator {
   private static Locator startLocator(int port, File logFile, InetAddress bindAddress,
       java.util.Properties dsProperties, boolean peerLocator, boolean serverLocator,
       String hostnameForClients) throws IOException {
-    return InternalLocator.startLocator(port, logFile, null, null, null, bindAddress, dsProperties,
-        hostnameForClients);
+    return InternalLocator.startLocator(port, logFile, null, null, null, bindAddress, true,
+        dsProperties, hostnameForClients);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
index 12c5c21..43ab546 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
@@ -649,7 +649,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
         // TODO : remove the extra param for loadFromSharedConfigDir
         try {
           this.locator = InternalLocator.startLocator(getPort(), getLogFile(), null, null, null,
-              getBindAddress(), getDistributedSystemProperties(), getHostnameForClients());
+              getBindAddress(), true, getDistributedSystemProperties(), getHostnameForClients());
         } finally {
           ProcessLauncherContext.remove();
         }

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 6500385..c299dd0 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -228,11 +228,13 @@ public class InternalLocator extends Locator implements ConnectListener {
       return false;
     }
     synchronized (locatorLock) {
-      if (hasLocator()) {
-        if (locator.equals(InternalLocator.locator)) {
-          InternalLocator.locator = null;
-          return true;
-        }
+      LogWriterAppenders.stop(LogWriterAppenders.Identifier.MAIN);
+      LogWriterAppenders.stop(LogWriterAppenders.Identifier.SECURITY);
+      LogWriterAppenders.destroy(LogWriterAppenders.Identifier.MAIN);
+      LogWriterAppenders.destroy(LogWriterAppenders.Identifier.SECURITY);
+      if (locator != null && locator.equals(InternalLocator.locator)) {
+        InternalLocator.locator = null;
+        return true;
       }
       return false;
     }
@@ -284,26 +286,6 @@ public class InternalLocator extends Locator implements ConnectListener {
   }
 
   /**
-   * Creates a distribution locator that runs in this VM on the given port and bind address and
-   * creates a distributed system.
-   * 
-   * @param port the tcp/ip port to listen on
-   * @param logFile the file that log messages should be written to
-   * @param logger a log writer that should be used (logFile parameter is ignored)
-   * @param securityLogger the logger to be used for security related log messages
-   * @param dsProperties optional properties to configure the distributed system (e.g., mcast
-   *        addr/port, other locators)
-   * @param hostnameForClients the name to give to clients for connecting to this locator
-   * @since GemFire 7.0
-   */
-  public static InternalLocator startLocator(int port, File logFile, File stateFile,
-      InternalLogWriter logger, InternalLogWriter securityLogger, InetAddress bindAddress,
-      Properties dsProperties, String hostnameForClients) throws IOException {
-    return startLocator(port, logFile, stateFile, logger, securityLogger, bindAddress, true,
-        dsProperties, hostnameForClients);
-  }
-
-  /**
    * Creates a distribution locator that runs in this VM on the given port and bind address.
    * <p>
    * This is for internal use only as it does not create a distributed system unless told to do so.
@@ -615,7 +597,8 @@ public class InternalLocator extends Locator implements ConnectListener {
       InternalLogWriter logger, InternalLogWriter logger1, InetAddress addr,
       Properties dsProperties, boolean peerLocator, boolean serverLocator, String s, boolean b1)
       throws IOException {
-    return startLocator(locatorPort, logFile, stateFile, logger, logger1, addr, dsProperties, s);
+    return startLocator(locatorPort, logFile, stateFile, logger, logger1, addr, true, dsProperties,
+        s);
   }
 
   class SharedConfigurationRunnable implements Runnable {

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
index f861515..e190d0b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
@@ -167,7 +167,7 @@ public class DistributionLocator {
       try {
 
         InternalLocator locator = InternalLocator.startLocator(port, new File(DEFAULT_LOG_FILE),
-            null, null, null, address, (Properties) null, hostnameForClients);
+            null, null, null, address, true, (Properties) null, hostnameForClients);
 
         ManagerInfo.setLocatorStarted(directory, port, address);
         locator.waitToStop();

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
index ca2de76..6d3f50f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
@@ -18,11 +18,11 @@ import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_SAM
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.execute.FunctionInvocationTargetException;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LogLevel;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
@@ -47,6 +47,7 @@ import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
+import org.apache.logging.log4j.Logger;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
@@ -72,6 +73,7 @@ public class ConfigCommands extends AbstractCommandsSupport {
       new GetMemberConfigInformationFunction();
   private final AlterRuntimeConfigFunction alterRunTimeConfigFunction =
       new AlterRuntimeConfigFunction();
+  private static Logger logger = LogService.getLogger();
 
   @CliCommand(value = {CliStrings.DESCRIBE_CONFIG}, help = CliStrings.DESCRIBE_CONFIG__HELP)
   @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
@@ -252,10 +254,10 @@ public class ConfigCommands extends AbstractCommandsSupport {
   public Result alterRuntimeConfig(
       @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__MEMBER},
           optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String memberNameOrId,
+          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String[] memberNameOrId,
       @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__GROUP},
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String group,
+          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String[] group,
       @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT},
           help = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT__HELP) Integer archiveDiskSpaceLimit,
       @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT},
@@ -287,153 +289,138 @@ public class ConfigCommands extends AbstractCommandsSupport {
 
     Map<String, String> runTimeDistributionConfigAttributes = new HashMap<>();
     Map<String, String> rumTimeCacheAttributes = new HashMap<>();
-    Set<DistributedMember> targetMembers;
-
-    try {
+    Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
 
-      targetMembers = CliUtil.findMembersOrThrow(group, memberNameOrId);
+    if (targetMembers.isEmpty()) {
+      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    }
 
-      if (archiveDiskSpaceLimit != null) {
-        runTimeDistributionConfigAttributes.put(
-            CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
-            archiveDiskSpaceLimit.toString());
-      }
+    if (archiveDiskSpaceLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
+          archiveDiskSpaceLimit.toString());
+    }
 
-      if (archiveFileSizeLimit != null) {
-        runTimeDistributionConfigAttributes.put(
-            CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
-            archiveFileSizeLimit.toString());
-      }
+    if (archiveFileSizeLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
+          archiveFileSizeLimit.toString());
+    }
 
-      if (logDiskSpaceLimit != null) {
-        runTimeDistributionConfigAttributes.put(
-            CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, logDiskSpaceLimit.toString());
-      }
+    if (logDiskSpaceLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, logDiskSpaceLimit.toString());
+    }
 
-      if (logFileSizeLimit != null) {
-        runTimeDistributionConfigAttributes.put(
-            CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, logFileSizeLimit.toString());
-      }
+    if (logFileSizeLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, logFileSizeLimit.toString());
+    }
 
-      if (logLevel != null && !logLevel.isEmpty()) {
-        runTimeDistributionConfigAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL,
-            logLevel);
-      }
+    if (logLevel != null && !logLevel.isEmpty()) {
+      runTimeDistributionConfigAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL,
+          logLevel);
+    }
 
-      if (statisticArchiveFile != null && !statisticArchiveFile.isEmpty()) {
-        runTimeDistributionConfigAttributes
-            .put(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, statisticArchiveFile);
-      }
+    if (statisticArchiveFile != null && !statisticArchiveFile.isEmpty()) {
+      runTimeDistributionConfigAttributes
+          .put(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, statisticArchiveFile);
+    }
 
-      if (statisticSampleRate != null) {
-        runTimeDistributionConfigAttributes.put(
-            CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE,
-            statisticSampleRate.toString());
-      }
+    if (statisticSampleRate != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, statisticSampleRate.toString());
+    }
 
-      if (statisticSamplingEnabled != null) {
-        runTimeDistributionConfigAttributes.put(STATISTIC_SAMPLING_ENABLED,
-            statisticSamplingEnabled.toString());
-      }
+    if (statisticSamplingEnabled != null) {
+      runTimeDistributionConfigAttributes.put(STATISTIC_SAMPLING_ENABLED,
+          statisticSamplingEnabled.toString());
+    }
 
 
-      // Attributes that are set on the cache.
-      if (setCopyOnRead != null) {
-        rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
-            setCopyOnRead.toString());
-      }
+    // Attributes that are set on the cache.
+    if (setCopyOnRead != null) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
+          setCopyOnRead.toString());
+    }
 
-      if (lockLease != null && lockLease > 0 && lockLease < Integer.MAX_VALUE) {
-        rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE,
-            lockLease.toString());
-      }
+    if (lockLease != null && lockLease > 0 && lockLease < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE,
+          lockLease.toString());
+    }
 
-      if (lockTimeout != null && lockTimeout > 0 && lockTimeout < Integer.MAX_VALUE) {
-        rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
-            lockTimeout.toString());
-      }
+    if (lockTimeout != null && lockTimeout > 0 && lockTimeout < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
+          lockTimeout.toString());
+    }
 
-      if (messageSyncInterval != null && messageSyncInterval > 0
-          && messageSyncInterval < Integer.MAX_VALUE) {
-        rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
-            messageSyncInterval.toString());
-      }
+    if (messageSyncInterval != null && messageSyncInterval > 0
+        && messageSyncInterval < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
+          messageSyncInterval.toString());
+    }
 
-      if (searchTimeout != null && searchTimeout > 0 && searchTimeout < Integer.MAX_VALUE) {
-        rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
-            searchTimeout.toString());
-      }
+    if (searchTimeout != null && searchTimeout > 0 && searchTimeout < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
+          searchTimeout.toString());
+    }
 
-      if (!runTimeDistributionConfigAttributes.isEmpty() || !rumTimeCacheAttributes.isEmpty()) {
-        Map<String, String> allRunTimeAttributes = new HashMap<>();
-        allRunTimeAttributes.putAll(runTimeDistributionConfigAttributes);
-        allRunTimeAttributes.putAll(rumTimeCacheAttributes);
-
-        ResultCollector<?, ?> rc = CliUtil.executeFunction(alterRunTimeConfigFunction,
-            allRunTimeAttributes, targetMembers);
-        List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
-        CompositeResultData crd = ResultBuilder.createCompositeResultData();
-        TabularResultData tabularData = crd.addSection().addTable();
-        Set<String> successfulMembers = new TreeSet<>();
-        Set<String> errorMessages = new TreeSet<>();
-
-
-        for (CliFunctionResult result : results) {
-          if (result.getThrowable() != null) {
-            errorMessages.add(result.getThrowable().getMessage());
-          } else {
-            successfulMembers.add(result.getMemberIdOrName());
-          }
-        }
-        final String lineSeparator = System.getProperty("line.separator");
-        if (!successfulMembers.isEmpty()) {
-          StringBuilder successMessageBuilder = new StringBuilder();
+    if (runTimeDistributionConfigAttributes.isEmpty() && rumTimeCacheAttributes.isEmpty()) {
+      return ResultBuilder
+          .createUserErrorResult(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE);
+    }
 
-          successMessageBuilder.append(CliStrings.ALTER_RUNTIME_CONFIG__SUCCESS__MESSAGE);
-          successMessageBuilder.append(lineSeparator);
+    Map<String, String> allRunTimeAttributes = new HashMap<>();
+    allRunTimeAttributes.putAll(runTimeDistributionConfigAttributes);
+    allRunTimeAttributes.putAll(rumTimeCacheAttributes);
 
-          for (String member : successfulMembers) {
-            successMessageBuilder.append(member);
-            successMessageBuilder.append(lineSeparator);
-          }
+    ResultCollector<?, ?> rc =
+        CliUtil.executeFunction(alterRunTimeConfigFunction, allRunTimeAttributes, targetMembers);
+    List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+    Set<String> successfulMembers = new TreeSet<>();
+    Set<String> errorMessages = new TreeSet<>();
 
-          Properties properties = new Properties();
-          properties.putAll(runTimeDistributionConfigAttributes);
-
-          Result result = ResultBuilder.createInfoResult(successMessageBuilder.toString());
-
-          // Set the Cache attributes to be modified
-          final XmlEntity xmlEntity = XmlEntity.builder().withType(CacheXml.CACHE)
-              .withAttributes(rumTimeCacheAttributes).build();
-          persistClusterConfiguration(result,
-              () -> getSharedConfiguration().modifyXmlAndProperties(properties, xmlEntity,
-                  group != null ? group.split(",") : null));
-          return result;
-        } else {
-          StringBuilder errorMessageBuilder = new StringBuilder();
-          errorMessageBuilder.append("Following errors occurred while altering runtime config");
-          errorMessageBuilder.append(lineSeparator);
-
-          for (String errorMessage : errorMessages) {
-            errorMessageBuilder.append(errorMessage);
-            errorMessageBuilder.append(lineSeparator);
-          }
-          return ResultBuilder.createUserErrorResult(errorMessageBuilder.toString());
-        }
+    for (CliFunctionResult result : results) {
+      if (result.getThrowable() != null) {
+        logger.info("Function failed: " + result.getThrowable());
+        errorMessages.add(result.getThrowable().getMessage());
       } else {
-        return ResultBuilder
-            .createUserErrorResult(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE);
+        successfulMembers.add(result.getMemberIdOrName());
       }
-    } catch (CommandResultException crex) {
-      return crex.getResult();
-    } catch (CacheClosedException e) {
-      return ResultBuilder.createGemFireErrorResult(e.getMessage());
-    } catch (FunctionInvocationTargetException e) {
-      return ResultBuilder.createGemFireErrorResult(CliStrings
-          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.ALTER_RUNTIME_CONFIG));
-    } catch (Exception e) {
-      return ResultBuilder.createGemFireErrorResult(
-          CliStrings.format(CliStrings.EXCEPTION_CLASS_AND_MESSAGE, e.getClass(), e.getMessage()));
+    }
+    final String lineSeparator = System.getProperty("line.separator");
+    if (!successfulMembers.isEmpty()) {
+      StringBuilder successMessageBuilder = new StringBuilder();
+
+      successMessageBuilder.append(CliStrings.ALTER_RUNTIME_CONFIG__SUCCESS__MESSAGE);
+      successMessageBuilder.append(lineSeparator);
+
+      for (String member : successfulMembers) {
+        successMessageBuilder.append(member);
+        successMessageBuilder.append(lineSeparator);
+      }
+
+      Properties properties = new Properties();
+      properties.putAll(runTimeDistributionConfigAttributes);
+
+      Result result = ResultBuilder.createInfoResult(successMessageBuilder.toString());
+
+      // Set the Cache attributes to be modified
+      final XmlEntity xmlEntity = XmlEntity.builder().withType(CacheXml.CACHE)
+          .withAttributes(rumTimeCacheAttributes).build();
+      persistClusterConfiguration(result,
+          () -> getSharedConfiguration().modifyXmlAndProperties(properties, xmlEntity, group));
+      return result;
+    } else {
+      StringBuilder errorMessageBuilder = new StringBuilder();
+      errorMessageBuilder.append("Following errors occurred while altering runtime config");
+      errorMessageBuilder.append(lineSeparator);
+
+      for (String errorMessage : errorMessages) {
+        errorMessageBuilder.append(errorMessage);
+        errorMessageBuilder.append(lineSeparator);
+      }
+      return ResultBuilder.createUserErrorResult(errorMessageBuilder.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/AlterRuntimeConfigFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/AlterRuntimeConfigFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/AlterRuntimeConfigFunction.java
index 1d3e5f5..53d3ab7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/AlterRuntimeConfigFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/AlterRuntimeConfigFunction.java
@@ -14,10 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.functions;
 
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.FunctionAdapter;
@@ -26,13 +22,21 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.ConfigSource;
 import org.apache.geode.internal.InternalEntity;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 
 public class AlterRuntimeConfigFunction extends FunctionAdapter implements InternalEntity {
 
   private static final long serialVersionUID = 1L;
 
+  private static Logger logger = LogService.getLogger();
+
   private InternalCache getCache() {
     return (InternalCache) CacheFactory.getAnyInstance();
   }
@@ -78,6 +82,7 @@ public class AlterRuntimeConfigFunction extends FunctionAdapter implements Inter
       context.getResultSender().lastResult(result);
 
     } catch (Exception e) {
+      logger.error("Exception happened on : " + memberId, e);
       CliFunctionResult cliFuncResult =
           new CliFunctionResult(memberId, e, CliUtil.stackTraceAsString(e));
       context.getResultSender().lastResult(cliFuncResult);

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalLocatorIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalLocatorIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalLocatorIntegrationTest.java
new file mode 100644
index 0000000..356c79f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/InternalLocatorIntegrationTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.distributed.internal;
+
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.internal.logging.log4j.LogWriterAppender;
+import org.apache.geode.internal.logging.log4j.LogWriterAppenders;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.Properties;
+
+@Category(IntegrationTest.class)
+public class InternalLocatorIntegrationTest {
+
+  private Locator locator;
+  private LogWriterAppender appender;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Test
+  public void testLogWriterAppenderShouldBeRemovedForALocatorWithNoDS() throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(NAME, "testVM");
+    properties.setProperty(LOG_FILE, temporaryFolder.newFile("testVM.log").getAbsolutePath());
+
+    int port = AvailablePortHelper.getRandomAvailableTCPPort();
+    locator =
+        InternalLocator.startLocator(port, null, null, null, null, null, false, properties, null);
+
+    appender = LogWriterAppenders.getAppender(LogWriterAppenders.Identifier.MAIN);
+    assertThat(appender).isNotNull();
+
+    locator.stop();
+
+    appender = LogWriterAppenders.getAppender(LogWriterAppenders.Identifier.MAIN);
+    assertThat(appender).isNull();
+  }
+
+  @Test
+  public void testLogWriterAppenderShouldBeRemovedForALocatorWithDS() throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(NAME, "testVM");
+    properties.setProperty(LOG_FILE, temporaryFolder.newFile("testVM.log").getAbsolutePath());
+
+    int port = AvailablePortHelper.getRandomAvailableTCPPort();
+    locator = InternalLocator.startLocatorAndDS(port, null, properties);
+
+    appender = LogWriterAppenders.getAppender(LogWriterAppenders.Identifier.MAIN);
+    assertThat(appender).isNotNull();
+
+    locator.stop();
+
+    appender = LogWriterAppenders.getAppender(LogWriterAppenders.Identifier.MAIN);
+    assertThat(appender).isNull();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
index edec00a..a110025 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
@@ -59,6 +59,7 @@ import org.apache.geode.management.internal.cli.remote.CommandProcessor;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
@@ -331,9 +332,10 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
         commandProcessor.createCommandStatement("alter runtime", Collections.EMPTY_MAP).process();
   }
 
-  @Category(FlakyTest.class) // GEODE-1313
   @Test
   public void testAlterRuntimeConfigRandom() throws Exception {
+    IgnoredException.addIgnoredException(
+        "java.lang.IllegalArgumentException: Could not set \"log-disk-space-limit\"");
     final String member1 = "VM1";
     final String controller = "controller";
 
@@ -352,7 +354,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
         Properties localProps = new Properties();
         localProps.setProperty(NAME, member1);
         getSystem(localProps);
-        Cache cache = getCache();
+        getCache();
       }
     });
 
@@ -360,9 +362,6 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     CommandResult cmdResult = executeCommand(csb.getCommandString());
     String resultAsString = commandResultToString(cmdResult);
 
-    getLogWriter().info("#SB Result\n");
-    getLogWriter().info(resultAsString);
-
     assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
     assertTrue(resultAsString.contains(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE));
 
@@ -371,10 +370,9 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     cmdResult = executeCommand(csb.getCommandString());
     resultAsString = commandResultToString(cmdResult);
 
-    getLogWriter().info("#SB Result\n");
-    getLogWriter().info(resultAsString);
-
     assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
+    assertTrue(
+        resultAsString.contains("Could not set \"log-disk-space-limit\" to \"2,000,000,000\""));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
index 055993c..dcdc5c4 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
@@ -75,8 +75,7 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     DUnitLauncher.closeAndCheckForSuspects();
     restoreSystemProperties.after();
     temporaryFolder.delete();
-    Arrays.stream(members).filter(Objects::nonNull)
-        .forEach(MemberVM::stopMemberAndCleanupVMIfNecessary);
+    Arrays.stream(members).filter(Objects::nonNull).forEach(MemberVM::stopMember);
   }
 
   public MemberVM<Locator> startLocatorVM(int index) throws Exception {
@@ -139,7 +138,7 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
 
   public void stopMember(int index) {
     MemberVM member = members[index];
-    member.stopMemberAndCleanupVMIfNecessary();
+    member.stopMember();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/ca12f781/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
index 6da824e..7e5ce1f 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
@@ -76,12 +76,9 @@ public class MemberVM<T extends Member> implements Member {
     return member.getName();
   }
 
-  public void stopMemberAndCleanupVMIfNecessary() {
-    stopMember();
-    cleanupVMIfNecessary();
-  }
+  public void stopMember() {
 
-  private void cleanupVMIfNecessary() {
+    this.invoke(LocatorServerStartupRule::stopMemberInThisVM);
     /**
      * The LocatorServerStarterRule may dynamically change the "user.dir" system property to point
      * to a temporary folder. The Path API caches the first value of "user.dir" that it sees, and
@@ -93,8 +90,4 @@ public class MemberVM<T extends Member> implements Member {
       this.getVM().bounce();
     }
   }
-
-  public void stopMember() {
-    this.invoke(LocatorServerStartupRule::stopMemberInThisVM);
-  }
 }