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 2020/10/28 13:55:52 UTC

[geode] branch develop updated: GEODE-8609: Create a dunit suspect file per VM (#5625)

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

jensdeppe pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 84ac086  GEODE-8609: Create a dunit suspect file per VM (#5625)
84ac086 is described below

commit 84ac0863ed4d08bb02c1a96982c89f8627994184
Author: Jens Deppe <jd...@pivotal.io>
AuthorDate: Wed Oct 28 06:55:00 2020 -0700

    GEODE-8609: Create a dunit suspect file per VM (#5625)
    
    - Create a new `dunit_suspect` log for each VM. The logs will now
      be named `dunit_suspect-vm<VMID>.log`. The locator VM and the test
      runner VM will have logs named `dunit_suspect-locator.log` and
      `dunit_suspect-local.log` respectively.
    - Leaving in the bad regexp which will be addressed by a subsequent Jira
---
 .../apache/geode/cache30/ReconnectDUnitTest.java   |   3 +-
 .../ClusterDistributionManagerDUnitTest.java       |   2 +-
 .../tier/sockets/RedundancyLevelTestBase.java      |   2 +-
 .../internal/security/MultiGfshDUnitTest.java      |  11 +-
 .../sockets/ClientServerMiscDUnitTestBase.java     |   6 +-
 .../cli/commands/QueryCommandDUnitTestBase.java    |  25 +---
 .../apache/geode/test/dunit/IgnoredException.java  |   3 +
 .../geode/test/dunit/internal/DUnitLauncher.java   | 166 ++++++++++++++-------
 .../test/dunit/internal/StandAloneDUnitEnv.java    |   2 +-
 .../apache/geode/test/greplogs/LogConsumer.java    |  32 ++--
 .../geode/test/greplogs/LogConsumerTest.java       |  24 +--
 .../AlterRuntimeCommandDistributedTest.java        |  59 +++-----
 12 files changed, 180 insertions(+), 155 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
index d8dbd26..2bfb534 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
@@ -134,6 +134,7 @@ public class ReconnectDUnitTest extends JUnit4CacheTestCase {
 
   @Override
   public final void postSetUp() throws Exception {
+    IgnoredException.addIgnoredException("ForcedDisconnectException||Possible loss of quorum");
     locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     final int locPort = locatorPort;
     Host.getHost(0).getVM(locatorVMNumber).invoke(new SerializableRunnable("start locator") {
@@ -148,8 +149,6 @@ public class ReconnectDUnitTest extends JUnit4CacheTestCase {
           system = (InternalDistributedSystem) locator.getDistributedSystem();
           cache = ((InternalLocator) locator).getCache();
           ReconnectDUnitTest.savedSystem = locator.getDistributedSystem();
-          IgnoredException.addIgnoredException(
-              "org.apache.geode.ForcedDisconnectException||Possible loss of quorum");
           // MembershipManagerHelper.getMembershipManager(InternalDistributedSystem.getConnectedInstance()).setDebugJGroups(true);
         } catch (IOException e) {
           Assert.fail("unable to start locator", e);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
index 30ceb66..3e3ab70 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
@@ -293,10 +293,10 @@ public class ClusterDistributionManagerDUnitTest extends CacheTestCase {
     assertThat(getCache().isClosed()).isFalse();
     Region<String, String> region = regionFactory.create("testRegion");
 
+    addIgnoredException("elapsed while waiting for replies");
     vm1.invoke("Connect to distributed system", () -> {
       config.setProperty(NAME, "sleeper");
       getSystem(config);
-      addIgnoredException("elapsed while waiting for replies");
 
       RegionFactory<String, String> regionFactory2 = getCache().createRegionFactory();
       regionFactory2.setScope(Scope.DISTRIBUTED_ACK);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RedundancyLevelTestBase.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RedundancyLevelTestBase.java
index 70952dd..4b2f1fd 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RedundancyLevelTestBase.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/RedundancyLevelTestBase.java
@@ -107,7 +107,7 @@ public class RedundancyLevelTestBase extends JUnit4DistributedTestCase {
     server2 = VM.getVM(2);
     server3 = VM.getVM(3);
 
-    IgnoredException.addIgnoredException("java.net.SocketException||java.net.ConnectException");
+    IgnoredException.addIgnoredException("java.net.SocketException|java.net.ConnectException");
 
     // start servers first
     PORT1 = server0.invoke(RedundancyLevelTestBase::createServerCache);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
index dcacb6f..c122fec 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiGfshDUnitTest.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 
 import java.util.List;
 
@@ -62,8 +61,6 @@ public class MultiGfshDUnitTest {
       while (true) {
         GfshCommandRule gfsh = new GfshCommandRule(server::getJmxPort, PortType.jmxManager);
         gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManager, "dataRead", "dataRead");
-
-        await();
         gfsh.close();
       }
     });
@@ -87,13 +84,15 @@ public class MultiGfshDUnitTest {
     });
 
 
-    VM vm3 = lsRule.getVM(3);
+    VM vm0 = lsRule.getVM(0);
     IgnoredException
-        .addIgnoredException("java.lang.IllegalArgumentException: Region does not exist: {0}", vm3);
-    IgnoredException.addIgnoredException("java.lang.ClassNotFoundException: myApp.myListener", vm3);
+        .addIgnoredException("java.lang.IllegalArgumentException: Region does not exist: RegionA",
+            vm0);
+    IgnoredException.addIgnoredException("java.lang.ClassNotFoundException: myApp.myListener", vm0);
 
     // set up vm_3 as another gfsh vm, and then connect as "super-user" and try to execute the
     // commands and assert we don't get a NotAuthorized Exception
+    VM vm3 = lsRule.getVM(3);
     AsyncInvocation vm3Invoke = vm3.invokeAsync("run as superUser", () -> {
       GfshCommandRule gfsh = new GfshCommandRule();
       gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManager, "data,cluster", "data,cluster");
diff --git a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
index ce06be8..a2bb445 100755
--- a/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscDUnitTestBase.java
@@ -19,6 +19,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.junit.Assert.assertEquals;
@@ -84,7 +85,6 @@ import org.apache.geode.logging.internal.OSProcess;
 import org.apache.geode.logging.internal.log4j.api.LogService;
 import org.apache.geode.test.dunit.DistributedTestUtils;
 import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
@@ -739,7 +739,7 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
   @Test
   public void testSystemCanBeCycledWithAnInitializedPool() {
     // work around GEODE-477
-    IgnoredException.addIgnoredException("Connection reset");
+    addIgnoredException("Connection reset");
     Properties props = new Properties();
     props.setProperty(MCAST_PORT, "0");
     props.setProperty(LOCATORS, "");
@@ -788,7 +788,7 @@ public class ClientServerMiscDUnitTestBase extends JUnit4CacheTestCase {
 
   @Test(expected = GemFireConfigException.class)
   public void clientIsPreventedFromConnectingToLocatorAsServer() {
-    IgnoredException.addIgnoredException("Improperly configured client detected");
+    addIgnoredException("Improperly configured client detected", VM.getLocator());
     ClientCacheFactory clientCacheFactory = new ClientCacheFactory();
     clientCacheFactory.addPoolServer("localhost", DistributedTestUtils.getDUnitLocatorPort());
     clientCacheFactory.setPoolSubscriptionEnabled(true);
diff --git a/geode-dunit/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommandDUnitTestBase.java b/geode-dunit/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommandDUnitTestBase.java
index 8a388b6..f321972 100644
--- a/geode-dunit/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommandDUnitTestBase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommandDUnitTestBase.java
@@ -53,7 +53,6 @@ import org.apache.geode.management.internal.cli.dto.Value1;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.model.ResultModel;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
@@ -121,17 +120,13 @@ public class QueryCommandDUnitTestBase {
 
   @Test
   public void testWithUnsetGfshEnvironmentVariables() {
-    IgnoredException ex =
-        addIgnoredException(QueryInvalidException.class.getSimpleName(), locator.getVM());
-    try {
-      String query =
-          "query --query=\"select ID , status , createTime , pk, floatMinValue from ${UNSET_REGION} where ID <= ${UNSET_PORTFOLIO_ID}"
-              + " and status=${UNSET_STATUS}" + "\" --interactive=false";
-      gfsh.executeAndAssertThat(query).statusIsError()
-          .containsOutput(String.format("Syntax error in query: %s", ""));
-    } finally {
-      ex.remove();
-    }
+    addIgnoredException(QueryInvalidException.class.getSimpleName(), locator.getVM());
+    String query =
+        "query --query=\"select ID , status , createTime , pk, floatMinValue from ${UNSET_REGION} "
+            + "where ID <= ${UNSET_PORTFOLIO_ID}"
+            + " and status=${UNSET_STATUS}" + "\" --interactive=false";
+    gfsh.executeAndAssertThat(query).statusIsError()
+        .containsOutput(String.format("Syntax error in query: %s", ""));
   }
 
   @Test
@@ -182,9 +177,7 @@ public class QueryCommandDUnitTestBase {
 
   @Test
   public void testQueryEvictedDataNotDeserializable() {
-    IgnoredException ex =
-        addIgnoredException(Exception.class.getSimpleName(), locator.getVM());
-
+    addIgnoredException(Exception.class.getSimpleName(), server1.getVM());
     server1.invoke(() -> setupReplicatedRegionWithEviction(DATA_REGION_WITH_EVICTION_NAME));
     locator.waitUntilRegionIsReadyOnExactlyThisManyServers(DATA_REGION_WITH_EVICTION_NAME_PATH, 1);
     server1
@@ -196,8 +189,6 @@ public class QueryCommandDUnitTestBase {
     validateSelectResult(commandResult, Boolean.FALSE, -1, new String[] {"Value"});
     assertThat(commandResult.asString())
         .contains("An IOException was thrown while deserializing");
-
-    ex.remove();
   }
 
   private static void prepareDataForRegion(String regionPath) {
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/IgnoredException.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/IgnoredException.java
index c9470f3..dacea84 100755
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/IgnoredException.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/IgnoredException.java
@@ -114,6 +114,9 @@ public class IgnoredException implements Serializable, AutoCloseable {
       vm.invoke(addRunnable);
     } else {
       Invoke.invokeInEveryVM(addRunnable);
+      if (Host.getLocator() != null) {
+        Invoke.invokeInLocator(addRunnable);
+      }
     }
 
     IGNORED_EXCEPTIONS.add(ignoredException);
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/DUnitLauncher.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/DUnitLauncher.java
index 7f08e2e..69cbd97 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/DUnitLauncher.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/DUnitLauncher.java
@@ -35,10 +35,12 @@ import java.lang.reflect.Method;
 import java.net.InetAddress;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
+import java.nio.file.Paths;
 import java.rmi.AlreadyBoundException;
 import java.rmi.NotBoundException;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.registry.Registry;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 import java.util.regex.Pattern;
@@ -108,8 +110,7 @@ public class DUnitLauncher {
   static final String STARTUP_TIMEOUT_MESSAGE =
       "VMs did not start up within " + STARTUP_TIMEOUT / 1000 + " seconds";
 
-  private static final String SUSPECT_FILENAME = "dunit_suspect.log";
-  private static File DUNIT_SUSPECT_FILE;
+  private static final String SUSPECT_FILENAME_PREFIX = "dunit_suspect";
 
   public static final String DUNIT_DIR = "dunit";
   public static final String WORKSPACE_DIR_PARAM = "WORKSPACE_DIR";
@@ -188,9 +189,8 @@ public class DUnitLauncher {
 
   private static void launch(boolean launchLocator) throws AlreadyBoundException, IOException,
       InterruptedException, NotBoundException {
-    DUNIT_SUSPECT_FILE = new File(SUSPECT_FILENAME);
-    DUNIT_SUSPECT_FILE.delete();
-    DUNIT_SUSPECT_FILE.deleteOnExit();
+
+    deleteDunitSuspectFiles();
 
     // create an RMI registry and add an object to share our tests config
     int namingPort = AvailablePortHelper.getRandomAvailableTCPPort();
@@ -259,8 +259,8 @@ public class DUnitLauncher {
    * format so that hydra will be able to parse them.
    */
   private static void addSuspectFileAppender(final String workspaceDir) {
-    final String suspectFilename = new File(workspaceDir, SUSPECT_FILENAME).getAbsolutePath();
-
+    final String suspectFilename = createDunitSuspectFile(DUnitEnv.get().getId(), workspaceDir)
+        .getAbsolutePath();
 
     Object mainLogger = LogManager.getLogger(LoggingProvider.MAIN_LOGGER_NAME);
     if (!(mainLogger instanceof org.apache.logging.log4j.core.Logger)) {
@@ -330,11 +330,7 @@ public class DUnitLauncher {
 
   public static void init(MasterRemote master) {
     DUnitEnv.set(new StandAloneDUnitEnv(master));
-    // fake out tests that are using a bunch of hydra stuff
-    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM);
-    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
-
-    addSuspectFileAppender(workspaceDir);
+    addSuspectFileAppender(getWorkspaceDir());
 
     // Free off heap memory when disconnecting from the distributed system
     System.setProperty(GEMFIRE_PREFIX + "free-off-heap-memory", "true");
@@ -343,61 +339,119 @@ public class DUnitLauncher {
     System.setProperty(LAUNCHED_PROPERTY, "true");
   }
 
+  private static List<File> getDunitSuspectFiles() {
+    File[] suspectFiles = getDunitSuspectsDir()
+        .listFiles((dir, name) -> name.startsWith(SUSPECT_FILENAME_PREFIX));
+
+    return Arrays.asList(suspectFiles);
+  }
+
+  private static File getDunitSuspectsDir() {
+    return Paths.get(getWorkspaceDir()).toFile();
+  }
+
+  private static void deleteDunitSuspectFiles() {
+    getDunitSuspectFiles().forEach(File::delete);
+  }
+
+  private static File createDunitSuspectFile(int vmId, String workingDir) {
+    String suffix;
+
+    switch (vmId) {
+      case -2:
+        suffix = "locator";
+        break;
+      case -1:
+        suffix = "local";
+        break;
+      default:
+        suffix = "vm" + vmId;
+    }
+
+    File dunitSuspect = new File(getDunitSuspectsDir(),
+        String.format("%s-%s.log", SUSPECT_FILENAME_PREFIX, suffix));
+    dunitSuspect.deleteOnExit();
+
+    return dunitSuspect;
+  }
+
+  private static String getWorkspaceDir() {
+    String workspaceDir = System.getProperty(DUnitLauncher.WORKSPACE_DIR_PARAM);
+    workspaceDir = workspaceDir == null ? new File(".").getAbsolutePath() : workspaceDir;
+
+    return workspaceDir;
+  }
+
   public static void closeAndCheckForSuspects() {
-    if (isLaunched()) {
-      final List<Pattern> expectedStrings = ExpectedStrings.create("dunit");
-      final LogConsumer logConsumer = new LogConsumer(true, expectedStrings, "log4j", 5);
+    if (!isLaunched()) {
+      return;
+    }
 
-      final StringBuilder suspectStringBuilder = new StringBuilder();
+    List<File> suspectFiles = getDunitSuspectFiles();
 
-      BufferedReader buffReader = null;
-      FileChannel fileChannel = null;
-      try {
-        fileChannel = new FileOutputStream(DUNIT_SUSPECT_FILE, true).getChannel();
-        buffReader = new BufferedReader(new FileReader(DUNIT_SUSPECT_FILE));
-      } catch (FileNotFoundException e) {
-        System.err.println("Could not find the suspect string output file: " + e);
-        return;
-      }
-      try {
-        String line;
-        try {
-          while ((line = buffReader.readLine()) != null) {
-            final StringBuilder builder = logConsumer.consume(line);
-            if (builder != null) {
-              suspectStringBuilder.append(builder);
-            }
-          }
-        } catch (IOException e) {
-          System.err.println("Could not read the suspect string output file: " + e);
-        }
+    if (suspectFiles.isEmpty()) {
+      throw new IllegalStateException("No dunit suspect log files found in '"
+          + getDunitSuspectsDir().getAbsolutePath()
+          + "' - perhaps a rule that is cleaning up before suspect processing has already run.");
+    }
 
-        try {
-          fileChannel.truncate(0);
-        } catch (IOException e) {
-          System.err.println("Could not truncate the suspect string output file: " + e);
-        }
+    StringBuilder suspectStringCollector = new StringBuilder();
+    for (File suspect : suspectFiles) {
+      checkSuspectFile(suspect, suspectStringCollector);
+    }
 
-      } finally {
-        try {
-          buffReader.close();
-          fileChannel.close();
-        } catch (IOException e) {
-          System.err.println("Could not close the suspect string output file: " + e);
+    if (suspectStringCollector.length() != 0) {
+      System.err.println("Suspicious strings were written to the log during this run.\n"
+          + "Fix the strings or use IgnoredException.addIgnoredException to ignore.\n"
+          + suspectStringCollector);
+
+      Assert.fail("Suspicious strings were written to the log during this run.\n"
+          + "Fix the strings or use IgnoredException.addIgnoredException to ignore.\n"
+          + suspectStringCollector);
+    }
+  }
+
+  private static void checkSuspectFile(File suspectFile, StringBuilder suspectStringCollector) {
+    final List<Pattern> expectedStrings = ExpectedStrings.create("dunit");
+    final LogConsumer logConsumer = new LogConsumer(true, expectedStrings,
+        suspectFile.getName(), 5);
+
+    BufferedReader buffReader;
+    FileChannel fileChannel;
+    try {
+      fileChannel = new FileOutputStream(suspectFile, true).getChannel();
+      buffReader = new BufferedReader(new FileReader(suspectFile));
+    } catch (FileNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+
+    try {
+      String line;
+      try {
+        while ((line = buffReader.readLine()) != null) {
+          final String suspectString = logConsumer.consume(line);
+          if (suspectString != null) {
+            suspectStringCollector.append(suspectString);
+          }
         }
+      } catch (IOException e) {
+        System.err.println("Could not read the suspect string output file: " + e);
       }
 
-      if (suspectStringBuilder.length() != 0) {
-        System.err.println("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use IgnoredException.addIgnoredException to ignore.\n"
-            + suspectStringBuilder);
+      try {
+        fileChannel.truncate(0);
+      } catch (IOException e) {
+        System.err.println("Could not truncate the suspect string output file: " + e);
+      }
 
-        Assert.fail("Suspicious strings were written to the log during this run.\n"
-            + "Fix the strings or use IgnoredException.addIgnoredException to ignore.\n"
-            + suspectStringBuilder);
+    } finally {
+      try {
+        buffReader.close();
+        fileChannel.close();
+      } catch (IOException e) {
+        System.err.println("Could not close the suspect string output file: " + e);
       }
     }
   }
 
-
 }
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/StandAloneDUnitEnv.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/StandAloneDUnitEnv.java
index ef8f416..ee6542b 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/StandAloneDUnitEnv.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/internal/StandAloneDUnitEnv.java
@@ -51,7 +51,7 @@ public class StandAloneDUnitEnv extends DUnitEnv {
 
   @Override
   public int getId() {
-    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1).intValue();
+    return Integer.getInteger(DUnitLauncher.VM_NUM_PARAM, -1);
   }
 
   @Override
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/greplogs/LogConsumer.java b/geode-dunit/src/main/java/org/apache/geode/test/greplogs/LogConsumer.java
index ab509a7..34fb8c9 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/greplogs/LogConsumer.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/greplogs/LogConsumer.java
@@ -74,7 +74,7 @@ public class LogConsumer {
     this.skipLimit = repeatLimit;
   }
 
-  public StringBuilder consume(CharSequence line) {
+  public String consume(CharSequence line) {
     lineNumber++;
 
     // IgnoredException injects lines into the log to start or end ignore periods.
@@ -131,7 +131,7 @@ public class LogConsumer {
     return null;
   }
 
-  public StringBuilder close() {
+  public String close() {
     if (saveFlag) {
       saveFlag = false;
       return enforceErrorLimit(1, all.toString(), savelinenum, logName);
@@ -183,7 +183,7 @@ public class LogConsumer {
     }
   }
 
-  private StringBuilder enforceErrorLimitsAtShortErrMatcher() {
+  private String enforceErrorLimitsAtShortErrMatcher() {
     // we found a blank line so print the suspect string and reset the savetag flag
     saveFlag = false;
 
@@ -270,26 +270,22 @@ public class LogConsumer {
         constantIgnoredPatterns.stream().anyMatch(p -> p.matcher(line).find());
   }
 
-  private StringBuilder enforceErrorLimit(int hits, String line, int linenum, String filename) {
+  private String enforceErrorLimit(int hits, String line, int linenum, String filename) {
     if (hits < skipLimit) {
-      StringBuilder string = new StringBuilder();
-      string
-          .append("-----------------------------------------------------------------------")
-          .append(lineSeparator())
-          .append("Found suspect string in ").append(filename).append(" at line ").append(linenum)
-          .append(lineSeparator()).append(lineSeparator())
-          .append(line).append(lineSeparator());
+      String string = "-----------------------------------------------------------------------"
+          + lineSeparator()
+          + "Found suspect string in '" + filename + "' at line " + linenum
+          + lineSeparator() + lineSeparator()
+          + line + lineSeparator();
       return string;
     }
 
     if (hits == skipLimit) {
-      StringBuilder string = new StringBuilder();
-      string
-          .append(lineSeparator()).append(lineSeparator())
-          .append("Hit occurrence limit of ").append(hits).append(" for this string.")
-          .append(lineSeparator())
-          .append("Further reporting of this type of error will be suppressed.")
-          .append(lineSeparator());
+      String string = lineSeparator() + lineSeparator()
+          + "Hit occurrence limit of " + hits + " for this string."
+          + lineSeparator()
+          + "Further reporting of this type of error will be suppressed."
+          + lineSeparator();
       return string;
     }
     return null;
diff --git a/geode-dunit/src/test/java/org/apache/geode/test/greplogs/LogConsumerTest.java b/geode-dunit/src/test/java/org/apache/geode/test/greplogs/LogConsumerTest.java
index d5476be..1380db6 100644
--- a/geode-dunit/src/test/java/org/apache/geode/test/greplogs/LogConsumerTest.java
+++ b/geode-dunit/src/test/java/org/apache/geode/test/greplogs/LogConsumerTest.java
@@ -50,14 +50,14 @@ public class LogConsumerTest {
 
   @Test
   public void consume_returnsNull_ifLineIsOk() {
-    StringBuilder value = logConsumer.consume("ok");
+    String value = logConsumer.consume("ok");
 
     assertThat(value).isNull();
   }
 
   @Test
   public void consume_returnsNull_ifLineIsEmpty() {
-    StringBuilder value = logConsumer.consume("");
+    String value = logConsumer.consume("");
 
     assertThat(value).isNull();
   }
@@ -74,7 +74,7 @@ public class LogConsumerTest {
   public void close_returnsNull_ifLineIsOk() {
     logConsumer.consume("ok");
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).isNull();
   }
@@ -83,7 +83,7 @@ public class LogConsumerTest {
   public void close_returnsNull_ifLineIsEmpty() {
     logConsumer.consume("");
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).isNull();
   }
@@ -93,7 +93,7 @@ public class LogConsumerTest {
     logConsumer.consume("[info 019/06/13 14:41:05.750 PDT <main> tid=0x1] " +
         NullPointerException.class.getName());
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).isNull();
   }
@@ -103,7 +103,7 @@ public class LogConsumerTest {
     String line = "[error 019/06/13 14:41:05.750 PDT <main> tid=0x1] message";
     logConsumer.consume(line);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).contains(line);
   }
@@ -112,7 +112,7 @@ public class LogConsumerTest {
   public void close_returnsNull_ifLineContains_warningLevelMessage() {
     logConsumer.consume("[warning 2019/06/13 14:41:05.750 PDT <main> tid=0x1] message");
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).isNull();
   }
@@ -122,7 +122,7 @@ public class LogConsumerTest {
     String line = "[fatal 2019/06/13 14:41:05.750 PDT <main> tid=0x1] message";
     logConsumer.consume(line);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).contains(line);
   }
@@ -132,7 +132,7 @@ public class LogConsumerTest {
     String line = "[severe 2019/06/13 14:41:05.750 PDT <main> tid=0x1] message";
     logConsumer.consume(line);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).contains(line);
   }
@@ -142,7 +142,7 @@ public class LogConsumerTest {
     String line = "[info 2019/06/13 14:41:05.750 PDT <main> tid=0x1] contains {}";
     logConsumer.consume(line);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).contains(line);
   }
@@ -152,7 +152,7 @@ public class LogConsumerTest {
     String line = "hydra.MasterDescription.master.locators={}";
     logConsumer.consume(line);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).isNull();
   }
@@ -161,7 +161,7 @@ public class LogConsumerTest {
   public void close_returnsLine_ifLineContainsException() {
     logConsumer.consume(EXCEPTION_MESSAGE);
 
-    StringBuilder value = logConsumer.close();
+    String value = logConsumer.close();
 
     assertThat(value).contains(EXCEPTION_MESSAGE);
   }
diff --git a/geode-web/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDistributedTest.java b/geode-web/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDistributedTest.java
index 522a510..22712e0 100644
--- a/geode-web/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDistributedTest.java
+++ b/geode-web/src/distributedTest/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeCommandDistributedTest.java
@@ -70,13 +70,11 @@ public class AlterRuntimeCommandDistributedTest {
   @Test
   @Parameters({"true", "false"})
   public void testAlterRuntimeConfig(boolean connectOverHttp) throws Exception {
-    ignoreIllegalArgumentException("Could not set \"log-disk-space-limit\"");
-
     MemberVM server0 = startupRule.startServerVM(0, x -> x
         .withJMXManager()
-        .withHttpService()
-        .withProperty(LOG_LEVEL, "error"));
+        .withHttpService());
 
+    ignoreIllegalArgumentException("Could not set \"log-disk-space-limit\"");
     if (connectOverHttp) {
       gfsh.connectAndVerify(server0.getHttpPort(), GfshCommandRule.PortType.http);
     } else {
@@ -334,12 +332,10 @@ public class AlterRuntimeCommandDistributedTest {
   @Parameters({"true", "false"})
   public void alterLogFileSizeLimit_changesConfigOnAllServers(boolean connectOverHttp)
       throws Exception {
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
-    MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
+    MemberVM server2 = startupRule.startServerVM(2, locator.getPort());
 
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
@@ -378,15 +374,12 @@ public class AlterRuntimeCommandDistributedTest {
   @Parameters({"true", "false"})
   public void alterLogFileSizeLimitNegative_errorCanNotSet(boolean connectOverHttp)
       throws Exception {
-    ignoreIllegalArgumentException("Could not set \"log-file-size-limit\"");
-
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
-    MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
+    MemberVM server2 = startupRule.startServerVM(2, locator.getPort());
 
+    ignoreIllegalArgumentException("Could not set \"log-file-size-limit\"");
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
     } else {
@@ -407,17 +400,15 @@ public class AlterRuntimeCommandDistributedTest {
   @Parameters({"true", "false"})
   public void alterLogFileSizeLimitTooBig_errorCanNotSet(boolean connectOverHttp)
       throws Exception {
-    ignoreIllegalArgumentException("Could not set \"log-file-size-limit\"");
-
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
 
+    Properties props = new Properties();
     props.setProperty(GROUPS, "G1");
     MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
 
+    ignoreIllegalArgumentException("Could not set \"log-file-size-limit\"");
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
     } else {
@@ -740,14 +731,12 @@ public class AlterRuntimeCommandDistributedTest {
   @Test
   @Parameters({"true", "false"})
   public void alterStatisticSampleRateRangeIsEnforced(boolean connectOverHttp) throws Exception {
-    ignoreIllegalArgumentException("Could not set \"statistic-sample-rate\"");
-
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
-    MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
+    MemberVM server2 = startupRule.startServerVM(2, locator.getPort());
+
+    ignoreIllegalArgumentException("Could not set \"statistic-sample-rate\"");
 
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
@@ -926,15 +915,12 @@ public class AlterRuntimeCommandDistributedTest {
   @Test
   @Parameters({"true", "false"})
   public void alterArchiveDiskSpaceLimitRangeIsEnforced(boolean connectOverHttp) throws Exception {
-    ignoreIllegalArgumentException("Could not set \"archive-disk-space-limit");
-
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
-    MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
+    MemberVM server2 = startupRule.startServerVM(2, locator.getPort());
 
+    ignoreIllegalArgumentException("Could not set \"archive-disk-space-limit");
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
     } else {
@@ -1133,15 +1119,12 @@ public class AlterRuntimeCommandDistributedTest {
   @Test
   @Parameters({"true", "false"})
   public void alterArchiveFileSizeLimitRangeIsEnforced(boolean connectOverHttp) throws Exception {
-    ignoreIllegalArgumentException("Could not set \"archive-file-size-limit\"");
-
-    Properties props = new Properties();
-    props.setProperty(LOG_LEVEL, "error");
 
     MemberVM locator = startupRule.startLocatorVM(0, l -> l.withHttpService());
-    MemberVM server1 = startupRule.startServerVM(1, props, locator.getPort());
-    MemberVM server2 = startupRule.startServerVM(2, props, locator.getPort());
+    MemberVM server1 = startupRule.startServerVM(1, locator.getPort());
+    MemberVM server2 = startupRule.startServerVM(2, locator.getPort());
 
+    ignoreIllegalArgumentException("Could not set \"archive-file-size-limit\"");
     if (connectOverHttp) {
       gfsh.connectAndVerify(locator.getHttpPort(), GfshCommandRule.PortType.http);
     } else {
@@ -1349,7 +1332,7 @@ public class AlterRuntimeCommandDistributedTest {
         DistributionConfig config = cache.getInternalDistributedSystem().getConfig();
 
         assertThat(config.getLogLevel())
-            .isEqualTo(LogWriterLevel.ERROR.intLevel());
+            .isEqualTo(LogWriterLevel.CONFIG.intLevel());
         assertThat(config.getLogFileSizeLimit())
             .isEqualTo(0);
         assertThat(config.getArchiveDiskSpaceLimit())