You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by ctubbsii <gi...@git.apache.org> on 2017/03/01 00:54:55 UTC

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

GitHub user ctubbsii opened a pull request:

    https://github.com/apache/accumulo/pull/223

    ACCUMULO-4409 Create AccumuloMonitorAppender

    Creates an AccumuloMonitorAppender, and does additional logging system cleanup in the files and scripts.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ctubbsii/accumulo ACCUMULO-4409-custom-appender

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/accumulo/pull/223.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #223
    
----
commit cac1a3f9cf6fa9a39ce5657328941c1794ce10a7
Author: Christopher Tubbs <ct...@apache.org>
Date:   2017-02-24T03:55:24Z

    ACCUMULO-4409 Create AccumuloMonitorAppender

commit 6af8bcb96f9b506043fc745406f8a13da011a2f1
Author: Christopher Tubbs <ct...@apache.org>
Date:   2017-02-28T23:43:21Z

    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.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/accumulo/pull/223


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103739634
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    --- End diff --
    
    Well, end of the day, this being a common-case for us in Accumulo, it seemed like something that would be good to consolidate, lest we accidentally forget the next time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103733722
  
    --- Diff: 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
    --- End diff --
    
    Yeah, it's just an example, though. I was mostly trying to minimize the number of system properties we were creating and using for the log4j config. We've already changed the name of the audit named logger, so its already a change in behavior. I think separating things out will help users track access to the various services. At this point, we've made it so users can easily do something different if they wish.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103705555
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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)
    --- End diff --
    
    With this added, I think default settings for `general.classpaths` should be removed from accumulo-site.xml and `accumulo-util create-config` should no longer configure classpaths.  I think we could have another section or appendix in the user manual for configuring Accumulo to use vendor-specific releases of Hadoop & Zookeeper.  There could be a comment here pointing users to that section if they are using a vendor release of Hadoop or Zookeeper. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103734610
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    --- End diff --
    
    Yes, I created a thread in the master which logged a message every 10 seconds, and killed and restarted the master on a few different ports. It's how I found that I needed to check for a change in the `modified zookeeper transaction id` instead of just a change in the value... otherwise it wouldn't update if the monitor was restarted at the same location.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103738878
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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"
    --- End diff --
    
    Ok, thanks for the explanation, guys.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r104052130
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    +      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[] newLocation, long newModifiedTransactionId) {
    +      // remove and close the last one, if it was non-null
    +      if (lastSocketAppender != null) {
    --- End diff --
    
    It won't be called multiple times with null. If `newLocation` is null, it means the ZK node didn't exist, and the modified transaction ID will be 0. So the code outside guards against, because the last modified transaction id is updated to 0 the first time `newLocation` is null.
    
    However, even if it were called multiple times, both the remove of the last appender and the close are both NOOPs if it has already been removed and closed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103718621
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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"
    --- End diff --
    
    This seems like a strange change to be included in this pull request..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103702178
  
    --- Diff: 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
    --- End diff --
    
    This is cool.  I am guessing `INHERITED` takes level of higher logger in tree. Is there documentation on this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103733316
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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"
    --- End diff --
    
    He's doing this because log4j tries to initialize his new appender and will throw a ClassNotFoundException as the Hadoop & Zookeeper jars set by `general.classpaths` are loaded later in Java by the Accumulo classloader.  While `general.classpaths` should available to users, it should be empty by default and the classpath required for Accumulo should be built in `accumulo-env.sh` before `java` command is called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on the issue:

    https://github.com/apache/accumulo/pull/223
  
    I tested this PR using a Mac and everything worked well!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103707238
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    --- End diff --
    
    Did you do any testing to verify that this class will switch to new monitor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103742009
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    --- End diff --
    
    Maybe. I just don't see much difference from remembering to do `new Daemon` instead of `new Thread` and remembering to do `setDaemon(true)`. The former adds a level of obfuscation that a developer has to track, but otherwise, they seem to be equivalent to me from the perspective of what the developer is responsible for remembering to do. The only way the `Daemon` class is better is if we had an executor service whose API explicitly wanted those instead of `Thread`. It's not the case here, but maybe it's the case elsewhere in Accumulo. I haven't looked.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103720637
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    --- End diff --
    
    No tests for this new code? :wink:


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103985292
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    +      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[] newLocation, long newModifiedTransactionId) {
    +      // remove and close the last one, if it was non-null
    +      if (lastSocketAppender != null) {
    --- End diff --
    
    When switchAppender is called multiple times with a null `newLocation` it will redo this work with the same `lastSocketAppender`.  Not sure if matters.  Is it ok to remove and close the same socket appender multiple times?  Will it cause spurious errors?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103735127
  
    --- Diff: assemble/bin/accumulo ---
    @@ -127,13 +127,12 @@ function main() {
         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=$ACCUMULO_CMD" \
    +  CLASSPATH="${conf}:${lib}/*:${CLASSPATH}"
    --- End diff --
    
    The common case would be to have this be non-null, since the rest of the class path will likely be set in the env file. This keeps it simple and readable, and extra colons in the class path don't cause issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103739428
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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)
    --- End diff --
    
    Building on the below Hadoop/ZK classpath building comment and the above "vendor-specific releases", I *think* this will be broken with what we have in HDP presently.
    
    Happy to work through it and be a "guinea pig" for how we want to support other layouts (avoiding naming the layout you're expecting now -- not sure if it's the "Apache Hadoop fs layout" or more correctly "Apache Bigtop layout", whatever)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103702937
  
    --- Diff: 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
    --- End diff --
    
    I am OK with this but it changes behaviour of audit log.  Previously, there was one audit log per node that was shared by all services.  This will create an audit log for each service.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103719146
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    --- End diff --
    
    Don't we have some `Daemon` class which encapsulates some of this logic already? (thread naming and doing `setDaemon(true)`)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103733871
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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)
    --- End diff --
    
    Agreed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103718100
  
    --- Diff: 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
    --- End diff --
    
    Hooray comments!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103732845
  
    --- Diff: 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
    --- End diff --
    
    Yes, log4j documents "INHERITED" and "NULL" as synonyms for the default behavior of inheriting from the parent logger. It'd also be fine if we left it as "INFO", but I figured it'd make more sense to default to the user needing to adjust it in fewer places if they wished to change it.
    
    INHERITED here means that it will inherit "INFO" from the root logger, but the `gui` appender is still going to filter out anything below "WARN".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103736418
  
    --- Diff: assemble/conf/templates/accumulo-env.sh ---
    @@ -53,20 +58,37 @@ 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=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)"
    -  "-Daccumulo.audit.log=$(hostname).audit")
    +  "-Daccumulo.application=${ACCUMULO_CMD}${ACCUMULO_SERVICE_INSTANCE}_$(hostname)")
     
     case "$ACCUMULO_CMD" in
    -monitor)                    JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-monitor.properties") ;;
    -gc|master|tserver|tracer)   JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/log4j-service.properties") ;;
    -*)                          JAVA_OPTS=("${JAVA_OPTS[@]}" "-Dlog4j.configuration=file:${ACCUMULO_CONF_DIR}/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"
    --- End diff --
    
    It was included in the second commit, where I switch to use the new appender, because the use of this appender requires some of these classes to be available during start-up.
    
    It's also something I've previously discussed doing as a sensible default: using standard java class path solutions in our launch scripts for bootstrap, rather than the xml configs with our custom class loading. This is easier for users to configure, more consistent with standard java applications, and easier for downstream packaging.
    
    I'd like to make further changes in this area, in issues more explicitly focused on that, but I needed to a little bit here, as a bare minimum, to get this new appender to work as a default.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103705868
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/Accumulo.java ---
    @@ -107,18 +104,6 @@ public static synchronized Path getAccumuloInstanceIdPath(VolumeManager fs) {
     
       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:")) {
    --- End diff --
    
    Good thing I didn't fix this :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103718041
  
    --- Diff: assemble/bin/accumulo ---
    @@ -127,13 +127,12 @@ function main() {
         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=$ACCUMULO_CMD" \
    +  CLASSPATH="${conf}:${lib}/*:${CLASSPATH}"
    --- End diff --
    
    Nit: do a `-z "${CLASSPATH}"` first? Saves that unnecessary colon in what would be the common case


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103738255
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    --- End diff --
    
    Not yet. I did a lot of manual testing. I was thinking about how to test it with a unit test, since Mike had explained why the previous MAC integration test wasn't that useful. I think I have a good idea of what that would look like, and will try to add it to the PR soon.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103739661
  
    --- Diff: 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
    --- End diff --
    
    Not sure why we even have "audit" logs in the shell. This seems non-sensical to me. The shell is a client. You could bypass auditing by simply tweaking the class path with a different log configuration before you run the shell.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103740195
  
    --- Diff: 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
    --- End diff --
    
    > Not sure why we even have "audit" logs in the shell. This seems non-sensical to me
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103740091
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    --- End diff --
    
    I was thinking some unit tests for these methods would be good, not necessarily high-level integration. Just some sanity checks that exercise your code given the state coming out of ZK.
    
    I'm sure there will be something wrong with what you have (seems to be the constant when working with ZK). Having some sort of boilerplate in place to test "given state X, observe change y, z should happen" would help better fix the inevitable issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103737450
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    --- End diff --
    
    Yes, we have a Daemon subclass of Thread, which is a trivial wrapper. I didn't feel the need to use it because I think it's better that developers can read this code without any knowledge or understanding of the behavior of some custom class.
    
    I'd probably favor deleting our Daemon class. I'm skeptical that it's all that useful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103739128
  
    --- Diff: 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
    --- End diff --
    
    I'm also curious if there would have been problems with different processes writing to the same file. I doubt the DailyRollingFileAppender used some sort of filesystem-independent lock mechanism to avoid corruption.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103981148
  
    --- Diff: 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
    --- End diff --
    
    Multiple processes writing to the same log file will not work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by mikewalch <gi...@git.apache.org>.
Github user mikewalch commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103706254
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/watcher/Log4jConfiguration.java ---
    @@ -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 {
    --- End diff --
    
    Nice to see this removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #223: ACCUMULO-4409 Create AccumuloMonitorAppender

Posted by ctubbsii <gi...@git.apache.org>.
Github user ctubbsii commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/223#discussion_r103742376
  
    --- Diff: server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.monitor.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ScheduledExecutorService;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import org.apache.accumulo.core.Constants;
    +import org.apache.accumulo.core.client.Instance;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.zookeeper.ZooUtil;
    +import org.apache.accumulo.fate.zookeeper.ZooCache;
    +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;
    +
    +public class AccumuloMonitorAppender extends AsyncAppender {
    +
    +  private final ScheduledExecutorService executorService;
    +  private final AtomicBoolean trackerScheduled;
    +
    +  /**
    +   * A Log4j Appender which follows the registered location of the active Accumulo monitor service, and forwards log messages to it
    +   */
    +  public AccumuloMonitorAppender() {
    +    // create the background thread to watch for updates to monitor location
    +    trackerScheduled = new AtomicBoolean(false);
    +    executorService = Executors.newSingleThreadScheduledExecutor(runnable -> {
    +      Thread t = new Thread(runnable, AccumuloMonitorAppender.class.getSimpleName() + " Location Tracker");
    +      t.setDaemon(true);
    +      return t;
    +    });
    +  }
    +
    +  @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)) {
    +      // wait 5 seconds, then run every 5 seconds
    +      executorService.scheduleAtFixedRate(new MonitorTracker(), 5, 5, TimeUnit.SECONDS);
    +    }
    +    super.activateOptions();
    +  }
    +
    +  @Override
    +  public void close() {
    +    if (!executorService.isShutdown()) {
    +      executorService.shutdownNow();
    +    }
    +    super.close();
    +  }
    +
    +  private class MonitorTracker implements Runnable {
    +
    +    private String path;
    +    private ZooCache zooCache;
    +
    +    private long lastModifiedTransactionId;
    +    private SocketAppender lastSocketAppender;
    +
    +    public MonitorTracker() {
    +
    +      // 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() {
    --- End diff --
    
    Agreed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---