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 2016/05/20 21:17:23 UTC

[4/5] incubator-geode git commit: GEODE-1415 Disable logging of banners in distributedTests

GEODE-1415 Disable logging of banners in distributedTests

Banners are normally logged during creation of an InternalDistributedSystem.
In DUnit tests the output is filled with these banners and they all have
the same information, so they are just clutter.

This change-set disables banners in non-locator VMs launched by
DUnitLauncher/ProcessManager.  Precheckin testing shows that the
amount of output for geode-core/distributedTests alone used to be over 1gb
and is now less than 500mb.


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

Branch: refs/heads/feature/GEODE-835
Commit: c674e35717c4f358ccfdfa04d65b0c3a65885342
Parents: 0014fd4
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri May 20 13:57:39 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri May 20 13:57:39 2016 -0700

----------------------------------------------------------------------
 .../internal/AdminDistributedSystemImpl.java    |  13 +-
 .../com/gemstone/gemfire/internal/Banner.java   | 252 +++++++++----------
 .../internal/logging/LogWriterFactory.java      |  14 +-
 ...rRemoteWithCustomLoggingIntegrationTest.java |   0
 .../DistributedSystemLogFileJUnitTest.java      |   2 +-
 .../test/dunit/standalone/DUnitLauncher.java    |   4 +
 .../test/dunit/standalone/ProcessManager.java   |   4 +
 7 files changed, 151 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
index 21a9ae3..2522a0f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.FutureTask;
 
+import com.gemstone.gemfire.distributed.internal.*;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelException;
@@ -56,10 +57,6 @@ import com.gemstone.gemfire.admin.SystemMembershipListener;
 import com.gemstone.gemfire.cache.persistence.PersistentID;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.FutureCancelledException;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.Banner;
@@ -215,8 +212,12 @@ implements com.gemstone.gemfire.admin.AdminDistributedSystem,
     } else {      
       // LOG: create LogWriterLogger
       this.logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config.createLogConfig(), false);
-      // LOG: changed statement from config to info
-      this.logWriter.info(Banner.getString(null));
+      if (!Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER)) {
+        // LOG: changed statement from config to info
+        this.logWriter.info(Banner.getString(null));
+      } else {
+        logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
+      }
       // Set this log writer in DistributedSystemConfigImpl
       this.config.setInternalLogWriter(this.logWriter);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
old mode 100644
new mode 100755
index 4b01e54..b6791ca
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
@@ -31,142 +31,142 @@ import java.util.*;
  */
 public class Banner {
 
-    private Banner() {
-	// everything is static so don't allow instance creation
+  private Banner() {
+    // everything is static so don't allow instance creation
+  }
+
+  private static void prettyPrintPath(String path, PrintWriter out) {
+    if (path != null) {
+      StringTokenizer st =
+          new StringTokenizer(path, System.getProperty("path.separator"));
+      while (st.hasMoreTokens()) {
+        out.println("  " + st.nextToken());
+      }
+    }
+  }
+  /**
+   * Print information about this process to the specified stream.
+   * @param args possibly null list of command line arguments
+   */
+  private static void print(PrintWriter out, String args[]) {
+    Map sp = new TreeMap((Properties)System.getProperties().clone()); // fix for 46822
+    int processId = -1;
+    final String SEPERATOR = "---------------------------------------------------------------------------";
+    try {
+      processId = OSProcess.getId();
+    }
+    catch (VirtualMachineError err) {
+      SystemFailure.initiateFailure(err);
+      // If this ever returns, rethrow the error.  We're poisoned
+      // now, so don't let this thread continue.
+      throw err;
     }
-    
-    private static void prettyPrintPath(String path, PrintWriter out) {
-	if (path != null) {
-	    StringTokenizer st =
-		new StringTokenizer(path, System.getProperty("path.separator"));
-	    while (st.hasMoreTokens()) {
-		out.println("  " + st.nextToken());
-	    }
-	}
+    catch (Throwable t) {
+      // Whenever you catch Error or Throwable, you must also
+      // catch VirtualMachineError (see above).  However, there is
+      // _still_ a possibility that you are dealing with a cascading
+      // error condition, so you also need to check to see if the JVM
+      // is still usable:
+      SystemFailure.checkFailure();
     }
-    /** 
-     * Print information about this process to the specified stream.
-     * @param args possibly null list of command line arguments
-     */
-    private static void print(PrintWriter out, String args[]) {
-        Map sp = new TreeMap((Properties)System.getProperties().clone()); // fix for 46822
-	int processId = -1;
-	final String SEPERATOR = "---------------------------------------------------------------------------";
-	try {
-	    processId = OSProcess.getId();
-	} 
-	catch (VirtualMachineError err) {
-	  SystemFailure.initiateFailure(err);
-	  // If this ever returns, rethrow the error.  We're poisoned
-	  // now, so don't let this thread continue.
-	  throw err;
-	}
-	catch (Throwable t) {
-	     // Whenever you catch Error or Throwable, you must also
-	     // catch VirtualMachineError (see above).  However, there is
-	     // _still_ a possibility that you are dealing with a cascading
-	     // error condition, so you also need to check to see if the JVM
-	     // is still usable:
-	     SystemFailure.checkFailure();
-	}
-	out.println();
+    out.println();
 
-        final String productName = GemFireVersion.getProductName();
-        
-	out.println(SEPERATOR);
+    final String productName = GemFireVersion.getProductName();
 
-        out.println("  ");
-        out.println("  Licensed to the Apache Software Foundation (ASF) under one or more");
-        out.println("  contributor license agreements.  See the NOTICE file distributed with this");
-        out.println("  work for additional information regarding copyright ownership.");
-        out.println("   ");
-        out.println("  The ASF licenses this file to You under the Apache License, Version 2.0");
-        out.println("  (the \"License\"); you may not use this file except in compliance with the");
-        out.println("  License.  You may obtain a copy of the License at");
-        out.println("  ");
-        out.println("  http://www.apache.org/licenses/LICENSE-2.0");
-        out.println("  ");
-        out.println("  Unless required by applicable law or agreed to in writing, software");
-        out.println("  distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT");
-        out.println("  WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.  See the");
-        out.println("  License for the specific language governing permissions and limitations");
-        out.println("  under the License.");
-        out.println("  ");
+    out.println(SEPERATOR);
 
-        out.println(SEPERATOR);
+    out.println("  ");
+    out.println("  Licensed to the Apache Software Foundation (ASF) under one or more");
+    out.println("  contributor license agreements.  See the NOTICE file distributed with this");
+    out.println("  work for additional information regarding copyright ownership.");
+    out.println("   ");
+    out.println("  The ASF licenses this file to You under the Apache License, Version 2.0");
+    out.println("  (the \"License\"); you may not use this file except in compliance with the");
+    out.println("  License.  You may obtain a copy of the License at");
+    out.println("  ");
+    out.println("  http://www.apache.org/licenses/LICENSE-2.0");
+    out.println("  ");
+    out.println("  Unless required by applicable law or agreed to in writing, software");
+    out.println("  distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT");
+    out.println("  WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.  See the");
+    out.println("  License for the specific language governing permissions and limitations");
+    out.println("  under the License.");
+    out.println("  ");
 
-	GemFireVersion.print(out);
+    out.println(SEPERATOR);
 
-	out.println("Process ID: " + processId);
-	out.println("User: " + sp.get("user.name"));
-        sp.remove("user.name");
-        sp.remove("os.name");
-        sp.remove("os.arch");
-	out.println("Current dir: " + sp.get("user.dir"));
-        sp.remove("user.dir");
-	out.println("Home dir: " + sp.get("user.home"));
-        sp.remove("user.home");
-        List<String> allArgs = new ArrayList<String>();
-        {
-          RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
-          if (runtimeBean != null) {
-            allArgs.addAll(runtimeBean.getInputArguments()); // fixes  45353
-          }
-        }
-        
-	if (args != null && args.length != 0) {
-	    for (int i=0; i < args.length; i++) {
-	      allArgs.add(args[i]);
-	    }
-	}
-	if (!allArgs.isEmpty()) {
-	  out.println("Command Line Parameters:");
-	  for (String arg: allArgs) {
-	    out.println("  " + arg);
-	  }
-	}
-	out.println("Class Path:");
-	prettyPrintPath((String)sp.get("java.class.path"), out);
-        sp.remove("java.class.path");
-	out.println("Library Path:");
-	prettyPrintPath((String)sp.get("java.library.path"), out);
-        sp.remove("java.library.path");
+    GemFireVersion.print(out);
+
+    out.println("Process ID: " + processId);
+    out.println("User: " + sp.get("user.name"));
+    sp.remove("user.name");
+    sp.remove("os.name");
+    sp.remove("os.arch");
+    out.println("Current dir: " + sp.get("user.dir"));
+    sp.remove("user.dir");
+    out.println("Home dir: " + sp.get("user.home"));
+    sp.remove("user.home");
+    List<String> allArgs = new ArrayList<String>();
+    {
+      RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+      if (runtimeBean != null) {
+        allArgs.addAll(runtimeBean.getInputArguments()); // fixes  45353
+      }
+    }
 
-        if (Boolean.getBoolean("gemfire.disableSystemPropertyLogging")) {
-          out.println("System property logging disabled.");
+    if (args != null && args.length != 0) {
+      for (int i=0; i < args.length; i++) {
+        allArgs.add(args[i]);
+      }
+    }
+    if (!allArgs.isEmpty()) {
+      out.println("Command Line Parameters:");
+      for (String arg: allArgs) {
+        out.println("  " + arg);
+      }
+    }
+    out.println("Class Path:");
+    prettyPrintPath((String)sp.get("java.class.path"), out);
+    sp.remove("java.class.path");
+    out.println("Library Path:");
+    prettyPrintPath((String)sp.get("java.library.path"), out);
+    sp.remove("java.library.path");
+
+    if (Boolean.getBoolean("gemfire.disableSystemPropertyLogging")) {
+      out.println("System property logging disabled.");
+    } else {
+      out.println("System Properties:");
+      Iterator it = sp.entrySet().iterator();
+      while (it.hasNext()) {
+        Map.Entry me = (Map.Entry)it.next();
+        String key = me.getKey().toString();
+        // SW: Filter out the security properties since they may contain
+        // sensitive information.
+        if (!key.startsWith(DistributionConfig.GEMFIRE_PREFIX
+            + DistributionConfig.SECURITY_PREFIX_NAME)
+            && !key.startsWith(DistributionConfigImpl.SECURITY_SYSTEM_PREFIX
+            + DistributionConfig.SECURITY_PREFIX_NAME)
+            && !key.toLowerCase().contains("password") /* bug 45381 */) {
+          out.println("    " + key + " = " + me.getValue());
         } else {
-	out.println("System Properties:");
-        Iterator it = sp.entrySet().iterator();
-        while (it.hasNext()) {
-          Map.Entry me = (Map.Entry)it.next();
-          String key = me.getKey().toString();
-          // SW: Filter out the security properties since they may contain
-          // sensitive information.
-          if (!key.startsWith(DistributionConfig.GEMFIRE_PREFIX
-              + DistributionConfig.SECURITY_PREFIX_NAME)
-              && !key.startsWith(DistributionConfigImpl.SECURITY_SYSTEM_PREFIX
-                  + DistributionConfig.SECURITY_PREFIX_NAME)
-              && !key.toLowerCase().contains("password") /* bug 45381 */) {
-            out.println("    " + key + " = " + me.getValue());
-          } else {
-            out.println("    " + key + " = " + "********");
-          }
+          out.println("    " + key + " = " + "********");
         }
-        out.println("Log4J 2 Configuration:");
-        out.println("    " + LogService.getConfigInformation());
-        }
-	out.println(SEPERATOR);
+      }
+      out.println("Log4J 2 Configuration:");
+      out.println("    " + LogService.getConfigInformation());
     }
+    out.println(SEPERATOR);
+  }
 
-    /**
-     * Return a string containing the banner information.
-     * @param args possibly null list of command line arguments
-     */
-    public static String getString(String args[]) {
-	StringWriter sw = new StringWriter();
-	PrintWriter pw = new PrintWriter(sw);
-	print(pw, args);
-	pw.close();
-	return sw.toString();
-    }
+  /**
+   * Return a string containing the banner information.
+   * @param args possibly null list of command line arguments
+   */
+  public static String getString(String args[]) {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    print(pw, args);
+    pw.close();
+    return sw.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
old mode 100644
new mode 100755
index d9352c0..63ed4b0
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
@@ -85,11 +85,15 @@ public class LogWriterFactory {
     }
     
     // log the banner
-    if (InternalDistributedSystem.getReconnectAttemptCounter() == 0 // avoid filling up logs during auto-reconnect
-        && !isSecure && (!isLoner /* do this on a loner to fix bug 35602 */
-        || !Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER))) {
-      // LOG:CONFIG:
-      logger.info(LogMarker.CONFIG, Banner.getString(null));
+    if (!Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER)) {
+      if (InternalDistributedSystem.getReconnectAttemptCounter() == 0 // avoid filling up logs during auto-reconnect
+          && !isSecure //&& !isLoner /* do this on a loner to fix bug 35602 */
+          ) {
+        // LOG:CONFIG:
+        logger.info(LogMarker.CONFIG, Banner.getString(null));
+      }
+    } else {
+      logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
     }
 
     // log the config

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
old mode 100644
new mode 100755
index 69df873..061ea97
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
@@ -127,7 +127,7 @@ public class DistributedSystemLogFileJUnitTest {
     // assert not empty
     FileInputStream fis = new FileInputStream(logFile);
     try {
-      assertTrue(fis.available() > 0);
+      assertTrue("log file is empty: " + logFile.getAbsoluteFile(), fis.available() > 0);
     } finally {
       fis.close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
index 0c600ab..8249996 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
@@ -38,6 +38,7 @@ import java.rmi.server.UnicastRemoteObject;
 import java.util.List;
 import java.util.Properties;
 
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.LoggerContext;
@@ -160,6 +161,9 @@ public class DUnitLauncher {
     master = new Master(registry, processManager);
     registry.bind(MASTER_PARAM, master);
 
+    // inhibit banners to make logs smaller
+    System.setProperty(InternalLocator.INHIBIT_DM_BANNER, "true");
+
     Runtime.getRuntime().addShutdownHook(new Thread() {
       public void run() {
 //        System.out.println("shutting down DUnit JVMs");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
old mode 100644
new mode 100755
index 5dbdfd6..1e78bff
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
@@ -33,6 +33,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import org.apache.commons.io.FileUtils;
 
 import com.gemstone.gemfire.internal.FileUtil;
@@ -169,6 +170,9 @@ public class ProcessManager {
     cmds.add("-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort);
     cmds.add("-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum);
     cmds.add("-D" + DUnitLauncher.WORKSPACE_DIR_PARAM + "=" + new File(".").getAbsolutePath());
+    if (vmNum >= 0) { // let the locator print a banner
+      cmds.add("-D" + InternalLocator.INHIBIT_DM_BANNER + "=true");
+    }
     cmds.add("-DlogLevel=" + DUnitLauncher.LOG_LEVEL);
     if (DUnitLauncher.LOG4J!=null) {
       cmds.add("-Dlog4j.configurationFile="+DUnitLauncher.LOG4J);