You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2017/03/09 01:53:21 UTC

[2/4] accumulo git commit: ACCUMULO-4409 More logging bootstrap cleanup

ACCUMULO-4409 More logging bootstrap cleanup

Clean up unneeded system properties, environment variables, logging
configuration, and related classes.

Fix AccumuloMonitorAppender to reload when monitor restarts in the same
location.


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

Branch: refs/heads/master
Commit: ff7525efea12c090a8c8fa9fa3d75f780cadee42
Parents: 9bc9ec3
Author: Christopher Tubbs <ct...@apache.org>
Authored: Tue Feb 28 18:43:21 2017 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Mar 6 17:15:26 2017 -0500

----------------------------------------------------------------------
 assemble/bin/accumulo                           |  10 +-
 assemble/conf/log4j-monitor.properties          |  14 +-
 assemble/conf/log4j-service.properties          |  38 ++---
 assemble/conf/log4j.properties                  |  12 +-
 assemble/conf/templates/accumulo-env.sh         |  51 ++++---
 .../org/apache/accumulo/server/Accumulo.java    |  17 +--
 .../server/watcher/Log4jConfiguration.java      |  59 -------
 .../server/watcher/MonitorLog4jWatcher.java     | 152 -------------------
 .../server/watcher/MonitorLog4jWatcherTest.java |  50 ------
 .../monitor/util/AccumuloMonitorAppender.java   |  70 ++++++---
 .../start/util/AsyncSocketAppender.java         | 111 --------------
 .../start/util/AsyncSocketAppenderTest.java     |  78 ----------
 12 files changed, 122 insertions(+), 540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/assemble/bin/accumulo
----------------------------------------------------------------------
diff --git a/assemble/bin/accumulo b/assemble/bin/accumulo
index b10d931..04b9836 100755
--- a/assemble/bin/accumulo
+++ b/assemble/bin/accumulo
@@ -111,14 +111,12 @@ function main() {
   if [ -n "$ACCUMULO_JAVA_PREFIX" ]; then
     JAVA=($ACCUMULO_JAVA_PREFIX $JAVA)
   fi
-  
-  CLASSPATH="${lib}/accumulo-start.jar:${conf}:${lib}/slf4j-api.jar:${lib}/slf4j-log4j12.jar:${lib}/log4j.jar:${CLASSPATH}"
 
-  exec "${JAVA[@]}" "-Dapp=$cmd" \
+  CLASSPATH="${conf}:${lib}/*:${CLASSPATH}"
+  export CLASSPATH
+
+  exec "${JAVA[@]}" \
      "${JAVA_OPTS[@]}" \
-     -classpath "${CLASSPATH}" \
-     -Dhadoop.home.dir="${HADOOP_PREFIX}" \
-     -Dzookeeper.home.dir="${ZOOKEEPER_HOME}" \
      org.apache.accumulo.start.Main \
      "$@"
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/assemble/conf/log4j-monitor.properties
----------------------------------------------------------------------
diff --git a/assemble/conf/log4j-monitor.properties b/assemble/conf/log4j-monitor.properties
index 8004677..acddc49 100644
--- a/assemble/conf/log4j-monitor.properties
+++ b/assemble/conf/log4j-monitor.properties
@@ -16,23 +16,21 @@
 ## Log4j 1.2 file that configures logging for Accumulo Monitor
 ## The system properties referenced below are configured by accumulo-env.sh
 
-## Write out INFO and higher to log file
+## Define a log file appender
 log4j.appender.file=org.apache.log4j.RollingFileAppender
-log4j.appender.file.File=${accumulo.log.dir}/${accumulo.service.id}.log
+log4j.appender.file.File=${accumulo.log.dir}/${accumulo.application}.log
 log4j.appender.file.MaxFileSize=100MB
 log4j.appender.file.MaxBackupIndex=10
 log4j.appender.file.Threshold=INFO
 log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 
-## Keep the last few log messages for display to the user in Monitor GUI
+## Define an appender for the Accumulo Monitor to log to its own web GUI
 log4j.appender.gui=org.apache.accumulo.server.monitor.LogService
-log4j.appender.gui.Keep=50
 log4j.appender.gui.Threshold=WARN
 
-## Log accumulo messages to file and gui appenders
-log4j.logger.org.apache.accumulo=INFO, file, gui
-log4j.additivity.org.apache.accumulo=false
+## Append monitor logs to its own web GUI
+log4j.logger.org.apache.accumulo=INHERITED, gui
 
-# Log non-accumulo messages to file
+## Append most logs to file
 log4j.rootLogger=INFO, file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/assemble/conf/log4j-service.properties
----------------------------------------------------------------------
diff --git a/assemble/conf/log4j-service.properties b/assemble/conf/log4j-service.properties
index e7212e0..bc01971 100644
--- a/assemble/conf/log4j-service.properties
+++ b/assemble/conf/log4j-service.properties
@@ -13,43 +13,47 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-## Log4j 1.2 file that configures logging for all Accumulo services (Master, TabletServer, GC & Tracer) except Monitor
+## Log4j 1.2 file that configures logging for all Accumulo services (Master, TabletServer, GC, and Tracer) except Monitor
 ## The system properties referenced below are configured by accumulo-env.sh
 
-## Write out INFO and higher to log file
+## Define a log file appender
 log4j.appender.file=org.apache.log4j.RollingFileAppender
-log4j.appender.file.File=${accumulo.log.dir}/${accumulo.service.id}.log
+log4j.appender.file.File=${accumulo.log.dir}/${accumulo.application}.log
 log4j.appender.file.MaxFileSize=100MB
 log4j.appender.file.MaxBackupIndex=10
 log4j.appender.file.Threshold=INFO
 log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 
-## Forwards Accumulo messages at WARN or higher to Accumulo Monitor
-log4j.appender.monitor=org.apache.accumulo.start.util.AsyncSocketAppender
-log4j.appender.monitor.RemoteHost=${accumulo.monitor.host}
-log4j.appender.monitor.Port=${accumulo.monitor.log.port}
-log4j.appender.monitor.Application=${accumulo.service.id}
+## Define an appender to send important logs to the the primary Accumulo Monitor
+## The primary monitor is the one currently holding a shared lock in ZooKeeper,
+## and is typically the one that started first.
+log4j.appender.monitor=org.apache.accumulo.monitor.util.AccumuloMonitorAppender
 log4j.appender.monitor.Threshold=WARN
 
-## Configures Audit logs which are OFF by default.
+## Uncomment to define a log file appender for audit logs
 #log4j.appender.audit=org.apache.log4j.DailyRollingFileAppender
-#log4j.appender.audit.File=${accumulo.log.dir}/${accumulo.audit.log}
+#log4j.appender.audit.File=${accumulo.log.dir}/${accumulo.application}.audit
 #log4j.appender.audit.DatePattern='.'yyyy-MM-dd
 #log4j.appender.audit.layout=org.apache.log4j.PatternLayout
-#log4j.appender.audit.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS/Z} [%c{2}] %-5p: %m%n
-#log4j.logger.org.apache.accumulo.audit=INFO, audit
+#log4j.appender.audit.layout.ConversionPattern=%d{ISO8601} [%c{2}] %-5p: %m%n
 #log4j.additivity.org.apache.accumulo.audit=false
-## Uncomment above and comment out line below to turn Audit logging ON
+
+## Change this log level from OFF to one of the following to enable audit logging:
+##   INFO
+##     enables audit logging (inherit appenders from root logger)
+##   INFO, audit
+##     enables audit logging using the audit log appender
+##     (requires audit log file appender above to be uncommented)
 log4j.logger.org.apache.accumulo.audit=OFF
 
-# Log Accumulo messages to file & monitor
-log4j.logger.org.apache.accumulo=INFO, file, monitor
-log4j.additivity.org.apache.accumulo=false
+## Append logs to the primary Accumulo Monitor
+log4j.logger.org.apache.accumulo=INHERITED, monitor
 
+## Constrain some particularly spammy loggers
 log4j.logger.org.apache.accumulo.core.file.rfile.bcfile=INFO
 log4j.logger.org.mortbay.log=WARN
 log4j.logger.org.apache.zookeeper=ERROR
 
-# Log non-accumulo messages to file
+## Append most logs to file
 log4j.rootLogger=INFO, file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/assemble/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/assemble/conf/log4j.properties b/assemble/conf/log4j.properties
index 5e602f2..1258fa6 100644
--- a/assemble/conf/log4j.properties
+++ b/assemble/conf/log4j.properties
@@ -13,22 +13,24 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Log4j 1.2 file that configures logging for Accumulo commands
+## Log4j 1.2 file that configures logging for clients and utility commands
 
-# Setup console appender
+## Define a console appender
 log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.Target=System.out
+log4j.appender.console.Threshold=ALL
 log4j.appender.console.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n
 log4j.appender.console.layout=org.apache.log4j.PatternLayout
 
-# hide AUDIT messages in the shell, alternatively you could send them to a different logger
+## Hide audit logs generated by the shell
 log4j.logger.org.apache.accumulo.shell.Shell.audit=WARN
 
+## Constrain some particularly spammy loggers
 log4j.logger.org.apache.accumulo.core.file.rfile.bcfile.Compression=WARN
-log4j.logger.org.apache.accumulo.test.TestRandomDeletes=WARN
 log4j.logger.org.apache.commons.vfs2.impl.DefaultFileSystemManager=WARN
 log4j.logger.org.apache.hadoop.io.compress=WARN
 log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.mortbay.log=WARN
 
-# By default, log everything at INFO or higher to the console
+## Append most logs to console
 log4j.rootLogger=INFO, console

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/assemble/conf/templates/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/assemble/conf/templates/accumulo-env.sh b/assemble/conf/templates/accumulo-env.sh
index a58d816..5e3864b 100644
--- a/assemble/conf/templates/accumulo-env.sh
+++ b/assemble/conf/templates/accumulo-env.sh
@@ -41,7 +41,7 @@ export ZOOKEEPER_HOME="${ZOOKEEPER_HOME:-/path/to/zookeeper}"
 ##################################################################
 
 ## JVM options set for all processes. Extra options can be passed in by setting ACCUMULO_JAVA_OPTS to an array of options.
-JAVA_OPTS=("${ACCUMULO_JAVA_OPTS[@]}" 
+JAVA_OPTS=("${ACCUMULO_JAVA_OPTS[@]}"
   '-XX:+UseConcMarkSweepGC'
   '-XX:CMSInitiatingOccupancyFraction=75'
   '-XX:+CMSClassUnloadingEnabled'
@@ -55,37 +55,54 @@ ${bin}/accumulo-util build-native &> /dev/null
 
 ## JVM options set for individual applications
 case "$cmd" in
-master)  JAVA_OPTS=("${JAVA_OPTS[@]}" ${masterHigh_masterLow}) ;;
-monitor) JAVA_OPTS=("${JAVA_OPTS[@]}" ${monitorHigh_monitorLow}) ;;
-gc)      JAVA_OPTS=("${JAVA_OPTS[@]}" ${gcHigh_gcLow}) ;;
-tserver) JAVA_OPTS=("${JAVA_OPTS[@]}" ${tServerHigh_tServerLow}) ;;
-shell)   JAVA_OPTS=("${JAVA_OPTS[@]}" ${shellHigh_shellLow}) ;;
-*)       JAVA_OPTS=("${JAVA_OPTS[@]}" ${otherHigh_otherLow}) ;;
+  master)  JAVA_OPTS=("${JAVA_OPTS[@]}" ${masterHigh_masterLow}) ;;
+  monitor) JAVA_OPTS=("${JAVA_OPTS[@]}" ${monitorHigh_monitorLow}) ;;
+  gc)      JAVA_OPTS=("${JAVA_OPTS[@]}" ${gcHigh_gcLow}) ;;
+  tserver) JAVA_OPTS=("${JAVA_OPTS[@]}" ${tServerHigh_tServerLow}) ;;
+  shell)   JAVA_OPTS=("${JAVA_OPTS[@]}" ${shellHigh_shellLow}) ;;
+  *)       JAVA_OPTS=("${JAVA_OPTS[@]}" ${otherHigh_otherLow}) ;;
 esac
 
-## JVM options set for logging.  Review logj4 properties files to see how they are used.
-JAVA_OPTS=("${JAVA_OPTS[@]}" 
+## JVM options set for logging. Review logj4 properties files to see how they are used.
+JAVA_OPTS=("${JAVA_OPTS[@]}"
   "-Daccumulo.log.dir=${ACCUMULO_LOG_DIR}"
-  "-Daccumulo.service.id=${cmd}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
-  "-Daccumulo.audit.log=$(hostname).audit")
+  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
 
 case "$cmd" in
-monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${conf}/log4j-monitor.properties") ;;
-gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${conf}/log4j-service.properties") ;;
-*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${conf}/log4j.properties") ;;
+  monitor)
+    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=log4j-monitor.properties")
+    ;;
+  gc|master|tserver|tracer)
+    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=log4j-service.properties")
+    ;;
+  *)
+    # let log4j use its default behavior (log4j.xml, log4j.properties)
+    true
+    ;;
 esac
 
 export JAVA_OPTS
 
+## External class path items for Java system class loader (dependencies not included with Accumulo)
+CLASSPATH="$(find "$ZOOKEEPER_HOME"/{,lib} "$HADOOP_PREFIX"/share/hadoop/{common,common/lib,hdfs,mapreduce,yarn} -maxdepth 1 -name '*.jar' \
+  -and -not -name '*slf4j*' \
+  -and -not -name '*fatjar*' \
+  -and -not -name '*-javadoc*' \
+  -and -not -name '*-sources*.jar' \
+  -and -not -name '*-test*.jar' \
+  -print0 | tr '\0' ':')$CLASSPATH"
+CLASSPATH="${HADOOP_CONF_DIR}:${CLASSPATH}"
+export CLASSPATH
+
 ############################
 # Variables set to a default
 ############################
 
 export MALLOC_ARENA_MAX=${MALLOC_ARENA_MAX:-1}
 ## Add Hadoop native libraries to shared library paths given operating system
-case "$(uname)" in 
-Darwin) export DYLD_LIBRARY_PATH="${HADOOP_PREFIX}/lib/native:${DYLD_LIBRARY_PATH}" ;; 
-*)      export LD_LIBRARY_PATH="${HADOOP_PREFIX}/lib/native:${LD_LIBRARY_PATH}" ;;
+case "$(uname)" in
+  Darwin) export DYLD_LIBRARY_PATH="${HADOOP_PREFIX}/lib/native:${DYLD_LIBRARY_PATH}" ;;
+  *)      export LD_LIBRARY_PATH="${HADOOP_PREFIX}/lib/native:${LD_LIBRARY_PATH}" ;;
 esac
 
 ###############################################

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index e423497..e849ae3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.File;
@@ -30,7 +31,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.AddressUtil;
@@ -44,7 +44,6 @@ import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.util.time.SimpleTimer;
-import org.apache.accumulo.server.watcher.MonitorLog4jWatcher;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,8 +51,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-
 public class Accumulo {
 
   private static final Logger log = Logger.getLogger(Accumulo.class);
@@ -107,18 +104,6 @@ public class Accumulo {
 
   public static void init(VolumeManager fs, ServerConfigurationFactory serverConfig, String application) throws IOException {
     final AccumuloConfiguration conf = serverConfig.getConfiguration();
-    final Instance instance = serverConfig.getInstance();
-
-    String logConfigFile = System.getProperty("log4j.configuration");
-    if (logConfigFile != null) {
-      if (logConfigFile.startsWith("file:")) {
-        logConfigFile = logConfigFile.split(":")[1];
-      }
-      // Set up polling log4j updates and log-forwarding using information advertised in zookeeper by the monitor
-      MonitorLog4jWatcher logConfigWatcher = new MonitorLog4jWatcher(instance.getInstanceID(), logConfigFile);
-      logConfigWatcher.setDelay(5000L);
-      logConfigWatcher.start();
-    }
 
     log.info(application + " starting");
     log.info("Instance " + serverConfig.getInstance().getInstanceID());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/server/base/src/main/java/org/apache/accumulo/server/watcher/Log4jConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/watcher/Log4jConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/watcher/Log4jConfiguration.java
deleted file mode 100644
index 68e4897..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/watcher/Log4jConfiguration.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.watcher;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.File;
-
-import org.apache.log4j.LogManager;
-import org.apache.log4j.PropertyConfigurator;
-import org.apache.log4j.xml.DOMConfigurator;
-
-/**
- * Encapsulate calls to PropertyConfigurator or DOMConfigurator to set up logging
- */
-public class Log4jConfiguration {
-
-  private final boolean usingProperties;
-  private final String filename;
-  private final File log4jFile;
-
-  public Log4jConfiguration(String filename) {
-    requireNonNull(filename, "log4j configuration filename must not be null");
-    usingProperties = filename.endsWith(".properties");
-    this.filename = filename;
-    log4jFile = new File(filename);
-  }
-
-  public boolean isUsingProperties() {
-    return usingProperties;
-  }
-
-  public void resetLogger() {
-    // Force a reset on the logger's configuration, but only if the configured log4j file actually exists
-    // If we reset the configuration blindly, the ITs will not get any logging as they don't set it up on their own
-    if (log4jFile.exists() && log4jFile.isFile() && log4jFile.canRead()) {
-      LogManager.resetConfiguration();
-      if (usingProperties) {
-        new PropertyConfigurator().doConfigure(filename, LogManager.getLoggerRepository());
-      } else {
-        new DOMConfigurator().doConfigure(filename, LogManager.getLoggerRepository());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java
deleted file mode 100644
index a0783a4..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.watcher;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.log4j.Appender;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.log4j.helpers.FileWatchdog;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-import com.google.common.net.HostAndPort;
-
-/**
- * Watcher that updates the monitor's log4j port from ZooKeeper in a system property
- */
-public class MonitorLog4jWatcher extends FileWatchdog implements Watcher {
-  private static final Logger log = Logger.getLogger(MonitorLog4jWatcher.class);
-
-  private static final String HOST_PROPERTY_NAME = "accumulo.monitor.host";
-  private static final String PORT_PROPERTY_NAME = "accumulo.monitor.log.port";
-
-  private final Object lock;
-  private final Log4jConfiguration logConfig;
-  private boolean loggingDisabled = false;
-  protected String path;
-
-  public MonitorLog4jWatcher(String instance, String filename) {
-    super(filename);
-    this.path = ZooUtil.getRoot(instance) + Constants.ZMONITOR_LOG4J_ADDR;
-    this.lock = new Object();
-    this.logConfig = new Log4jConfiguration(filename);
-    doOnChange();
-  }
-
-  boolean isUsingProperties() {
-    return logConfig.isUsingProperties();
-  }
-
-  String getPath() {
-    return path;
-  }
-
-  @Override
-  public void run() {
-    try {
-      // Initially set the logger if the Monitor's log4j advertisement node exists
-      if (ZooReaderWriter.getInstance().exists(path, this))
-        updateMonitorLog4jLocation();
-      log.info("Set watch for Monitor Log4j watcher");
-    } catch (Exception e) {
-      log.error("Unable to set watch for Monitor Log4j watcher on " + path);
-    }
-
-    super.run();
-  }
-
-  @Override
-  public void doOnChange() {
-    // this method gets called in the parent class' constructor
-    // I'm not sure of a better way to get around this. The final modifier helps though.
-    if (null == lock) {
-      return;
-    }
-
-    synchronized (lock) {
-      // We might triggered by file-reloading or from ZK update.
-      // Either way will result in log-forwarding being restarted
-      loggingDisabled = false;
-      log.info("Enabled log-forwarding");
-      logConfig.resetLogger();
-    }
-  }
-
-  @Override
-  public void process(WatchedEvent event) {
-    // We got an update, process the data in the node
-    updateMonitorLog4jLocation();
-
-    if (event.getPath() != null) {
-      try {
-        ZooReaderWriter.getInstance().exists(event.getPath(), this);
-      } catch (Exception ex) {
-        log.error("Unable to reset watch for Monitor Log4j watcher", ex);
-      }
-    }
-  }
-
-  /**
-   * Read the host and port information for the Monitor's log4j socket and update the system properties so that, on logger refresh, it sees the new information.
-   */
-  protected void updateMonitorLog4jLocation() {
-    try {
-      String hostPortString = new String(ZooReaderWriter.getInstance().getData(path, null), UTF_8);
-      HostAndPort hostAndPort = HostAndPort.fromString(hostPortString);
-
-      System.setProperty(HOST_PROPERTY_NAME, hostAndPort.getHostText());
-      System.setProperty(PORT_PROPERTY_NAME, Integer.toString(hostAndPort.getPort()));
-
-      log.info("Changing monitor log4j address to " + hostAndPort.toString());
-
-      doOnChange();
-    } catch (NoNodeException e) {
-      // Not sure on the synchronization guarantees for Loggers and Appenders
-      // on configuration reload
-      synchronized (lock) {
-        // Don't need to try to re-disable'ing it.
-        if (loggingDisabled) {
-          return;
-        }
-
-        Logger logger = LogManager.getLogger("org.apache.accumulo");
-        if (null != logger) {
-          // TODO ACCUMULO-2343 Create a specific appender for log-forwarding to the monitor
-          // that can replace the AsyncAppender+SocketAppender.
-          Appender appender = logger.getAppender("ASYNC");
-          if (null != appender) {
-            log.info("Closing log-forwarding appender");
-            appender.close();
-            log.info("Removing log-forwarding appender");
-            logger.removeAppender(appender);
-            loggingDisabled = true;
-          }
-        }
-      }
-    } catch (IllegalArgumentException e) {
-      log.error("Could not parse host and port information", e);
-    } catch (Exception e) {
-      log.error("Error reading zookeeper data for Monitor Log4j watcher", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java b/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
deleted file mode 100644
index 53c3c6e..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.watcher;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.junit.Before;
-import org.junit.Test;
-
-public class MonitorLog4jWatcherTest {
-  private static final String INSTANCE = "instance";
-  private static final String FILENAME = "something_logger.xml";
-
-  private MonitorLog4jWatcher w;
-
-  @Before
-  public void setUp() throws Exception {
-    w = new MonitorLog4jWatcher(INSTANCE, FILENAME);
-  }
-
-  @Test
-  public void testGetters() {
-    assertFalse(w.isUsingProperties());
-    assertEquals(ZooUtil.getRoot(INSTANCE) + Constants.ZMONITOR_LOG4J_ADDR, w.getPath());
-  }
-
-  @Test
-  public void testPropertyDetection() {
-    w = new MonitorLog4jWatcher(INSTANCE, FILENAME + ".properties");
-    assertTrue(w.isUsingProperties());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
index 5802920..8a855d0 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.monitor.util;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.util.Arrays;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -33,6 +32,7 @@ import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.AsyncAppender;
 import org.apache.log4j.net.SocketAppender;
+import org.apache.zookeeper.data.Stat;
 
 import com.google.common.net.HostAndPort;
 
@@ -48,7 +48,7 @@ public class AccumuloMonitorAppender extends AsyncAppender {
     // create the background thread to watch for updates to monitor location
     trackerScheduled = new AtomicBoolean(false);
     executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
-      Thread t = new Thread(runnable, "MonitorLog4jLocationTracker");
+      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
       t.setDaemon(true);
       return t;
     });
@@ -56,8 +56,10 @@ public class AccumuloMonitorAppender extends AsyncAppender {
 
   @Override
   public void activateOptions() {
+    // only schedule it once (in case options get activated more than once); not sure if this is possible
     if (trackerScheduled.compareAndSet(false, true)) {
-      executorService.scheduleAtFixedRate(new MonitorTracker(HdfsZooInstance.getInstance()), 5, 10, TimeUnit.SECONDS);
+      // wait 5 seconds, then run every 5 seconds
+      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
     }
     super.activateOptions();
   }
@@ -72,52 +74,78 @@ public class AccumuloMonitorAppender extends AsyncAppender {
 
   private class MonitorTracker implements Runnable {
 
-    private final String path;
-    private final ZooCache zooCache;
+    private String path;
+    private ZooCache zooCache;
 
-    private byte[] lastLocation;
+    private long lastModifiedTransactionId;
     private SocketAppender lastSocketAppender;
 
-    public MonitorTracker(Instance instance) {
-      this.path = ZooUtil.getRoot(instance) + Constants.ZMONITOR_LOG4J_ADDR;
-      this.zooCache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    public MonitorTracker() {
 
-      this.lastLocation = null;
+      // path and zooCache are lazily set the first time this tracker is run
+      // this allows the tracker to be constructed and scheduled during log4j initialization without
+      // triggering any actual logs from the Accumulo or ZooKeeper code
+      this.path = null;
+      this.zooCache = null;
+
+      this.lastModifiedTransactionId = 0;
       this.lastSocketAppender = null;
     }
 
     @Override
     public void run() {
-      byte[] loc = zooCache.get(path);
-      if (!Arrays.equals(loc, lastLocation)) {
-        // something changed
-        switchAppender(loc);
+      try {
+        // lazily set up path and zooCache (see comment in constructor)
+        if (this.zooCache == null) {
+          Instance instance = HdfsZooInstance.getInstance();
+          this.path = ZooUtil.getRoot(instance) + Constants.ZMONITOR_LOG4J_ADDR;
+          this.zooCache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+        }
+
+        // get the current location from the cache and update if necessary
+        Stat stat = new Stat();
+        byte[] loc = zooCache.get(path, stat);
+        long modifiedTransactionId = stat.getMzxid();
+
+        // modifiedTransactionId will be 0 if no current location
+        // lastModifiedTransactionId will be 0 if we've never seen a location
+        if (modifiedTransactionId != lastModifiedTransactionId) {
+          // replace old socket on every change, even if new location is the same as old location
+          // if modifiedTransactionId changed, then the monitor restarted and the old socket is dead now
+          switchAppender(loc, modifiedTransactionId);
+        }
+      } catch (Exception e) {
+        // dump any non-fatal problems to the console, but let it run again
+        e.printStackTrace();
       }
     }
 
-    private void switchAppender(byte[] loc) {
+    private void switchAppender(byte[] newLocation, long newModifiedTransactionId) {
       // remove and close the last one, if it was non-null
       if (lastSocketAppender != null) {
         AccumuloMonitorAppender.this.removeAppender(lastSocketAppender);
         lastSocketAppender.close();
       }
 
-      // create a new one, if it is non-null
-      if (loc != null) {
+      // create a new SocketAppender, if new location is non-null
+      if (newLocation != null) {
 
         int defaultPort = Integer.parseUnsignedInt(Property.MONITOR_LOG4J_PORT.getDefaultValue());
-        HostAndPort remote = HostAndPort.fromString(new String(loc, UTF_8));
+        HostAndPort remote = HostAndPort.fromString(new String(newLocation, UTF_8));
 
         SocketAppender socketAppender = new SocketAppender();
+        socketAppender.setApplication(System.getProperty("accumulo.application", "unknown"));
         socketAppender.setRemoteHost(remote.getHostText());
         socketAppender.setPort(remote.getPortOrDefault(defaultPort));
 
-        lastLocation = loc;
-        lastSocketAppender = socketAppender;
-
         socketAppender.activateOptions();
         AccumuloMonitorAppender.this.addAppender(socketAppender);
+
+        lastSocketAppender = socketAppender;
       }
+
+      // update lastModifiedTransactionId, even if the new one is 0 (no new location)
+      lastModifiedTransactionId = newModifiedTransactionId;
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/start/src/main/java/org/apache/accumulo/start/util/AsyncSocketAppender.java
----------------------------------------------------------------------
diff --git a/start/src/main/java/org/apache/accumulo/start/util/AsyncSocketAppender.java b/start/src/main/java/org/apache/accumulo/start/util/AsyncSocketAppender.java
deleted file mode 100644
index db64ecb..0000000
--- a/start/src/main/java/org/apache/accumulo/start/util/AsyncSocketAppender.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.start.util;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.log4j.AsyncAppender;
-import org.apache.log4j.net.SocketAppender;
-import org.apache.log4j.spi.LoggingEvent;
-
-/**
- * An asynchronous appender that maintains its own internal socket appender. Unlike <code>AsyncAppender</code>, this appender can be configured with a Log4J
- * properties file, although in that case no additional appenders can be added.
- */
-public class AsyncSocketAppender extends AsyncAppender {
-
-  private final SocketAppender socketAppender;
-  private final AtomicBoolean activated = new AtomicBoolean(false);
-
-  /**
-   * Creates a new appender.
-   */
-  public AsyncSocketAppender() {
-    socketAppender = new SocketAppender();
-  }
-
-  /**
-   * Creates a new appender using the given socket appender internally. Use this constructor for testing only.
-   */
-  AsyncSocketAppender(SocketAppender socketAppender) {
-    this.socketAppender = socketAppender;
-  }
-
-  @Override
-  public void append(final LoggingEvent event) {
-    // Lazy attachment, to avoid calling non-final method in constructor
-    if (!isAttached(socketAppender)) {
-      addAppender(socketAppender);
-    }
-
-    // Lazy activation / connection too, to allow setting host and port
-    if (activated.compareAndSet(false, true)) {
-      socketAppender.activateOptions();
-    }
-
-    super.append(event);
-  }
-
-  // SocketAppender delegate methods
-
-  public String getApplication() {
-    return socketAppender.getApplication();
-  }
-
-  // super.getLocationInfo() will always agree with socketAppender
-  public int getPort() {
-    return socketAppender.getPort();
-  }
-
-  public int getReconnectionDelay() {
-    return socketAppender.getReconnectionDelay();
-  }
-
-  public String getRemoteHost() {
-    return socketAppender.getRemoteHost();
-  }
-
-  public boolean isAdvertiseViaMulticastDNS() {
-    return socketAppender.isAdvertiseViaMulticastDNS();
-  }
-
-  public void setAdvertiseViaMulticastDNS(boolean advertiseViaMulticastDNS) {
-    socketAppender.setAdvertiseViaMulticastDNS(advertiseViaMulticastDNS);
-  }
-
-  public void setApplication(String lapp) {
-    socketAppender.setApplication(lapp);
-  }
-
-  @Override
-  public void setLocationInfo(boolean locationInfo) {
-    super.setLocationInfo(locationInfo);
-    socketAppender.setLocationInfo(locationInfo);
-  }
-
-  public void setPort(int port) {
-    socketAppender.setPort(port);
-  }
-
-  public void setReconnectionDelay(int delay) {
-    socketAppender.setReconnectionDelay(delay);
-  }
-
-  public void setRemoteHost(String host) {
-    socketAppender.setRemoteHost(host);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff7525ef/start/src/test/java/org/apache/accumulo/start/util/AsyncSocketAppenderTest.java
----------------------------------------------------------------------
diff --git a/start/src/test/java/org/apache/accumulo/start/util/AsyncSocketAppenderTest.java b/start/src/test/java/org/apache/accumulo/start/util/AsyncSocketAppenderTest.java
deleted file mode 100644
index 6c73782..0000000
--- a/start/src/test/java/org/apache/accumulo/start/util/AsyncSocketAppenderTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.start.util;
-
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.log4j.net.SocketAppender;
-import org.apache.log4j.spi.LoggingEvent;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class AsyncSocketAppenderTest {
-  private SocketAppender sa;
-  private AsyncSocketAppender asa;
-
-  @Before
-  public void setUp() throws Exception {
-    sa = EasyMock.createMock(SocketAppender.class);
-  }
-
-  @Test
-  public void testDelegates() {
-    asa = new AsyncSocketAppender();
-    asa.setApplication("myapp");
-    asa.setLocationInfo(true);
-    asa.setPort(1234);
-    asa.setReconnectionDelay(56);
-    asa.setRemoteHost("remotehost");
-    Assert.assertEquals("myapp", asa.getApplication());
-    Assert.assertEquals(true, asa.getLocationInfo()); // not really delegating
-    Assert.assertEquals(1234, asa.getPort());
-    Assert.assertEquals(56, asa.getReconnectionDelay());
-    Assert.assertEquals("remotehost", asa.getRemoteHost());
-  }
-
-  @Test
-  public void testSetLocationInfo() {
-    sa.setLocationInfo(true);
-    EasyMock.replay(sa);
-    asa = new AsyncSocketAppender(sa);
-    asa.setLocationInfo(true);
-    EasyMock.verify(sa);
-  }
-
-  @Test
-  public void testAppend() {
-    asa = new AsyncSocketAppender(sa);
-    Assert.assertFalse(asa.isAttached(sa));
-    LoggingEvent event1 = new LoggingEvent("java.lang.String", Logger.getRootLogger(), Level.INFO, "event1", null);
-    LoggingEvent event2 = new LoggingEvent("java.lang.Integer", Logger.getRootLogger(), Level.WARN, "event2", null);
-    sa.activateOptions();
-    sa.doAppend(event1);
-    sa.doAppend(event2);
-    sa.close();
-    EasyMock.replay(sa);
-    asa.doAppend(event1);
-    asa.doAppend(event2);
-    asa.close(); // forces events to be appended to socket appender
-    Assert.assertTrue(asa.isAttached(sa));
-    EasyMock.verify(sa);
-  }
-}