Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E25A51177B for ; Mon, 5 May 2014 13:39:36 +0000 (UTC) Received: (qmail 86429 invoked by uid 500); 5 May 2014 13:39:35 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 86354 invoked by uid 500); 5 May 2014 13:39:34 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 86325 invoked by uid 99); 5 May 2014 13:39:33 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 05 May 2014 13:39:33 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 499B690DCFD; Mon, 5 May 2014 13:39:33 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: bhavanki@apache.org To: commits@accumulo.apache.org Date: Mon, 05 May 2014 13:39:33 -0000 Message-Id: <41f3cf3754fe4a83918c19edd084acc8@git.apache.org> In-Reply-To: <04a1688d9145416fa7df66119e1217c2@git.apache.org> References: <04a1688d9145416fa7df66119e1217c2@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/3] git commit: ACCUMULO-2383 Add log4j properties support ACCUMULO-2383 Add log4j properties support Accumulo now looks for either XML or properties files for the Log4J configuration. It will also heed the log4j.configuration isystem property if it is set. Its MonitorLog4jWatcher can now load either properties or XML. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/cdcc1d4d Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/cdcc1d4d Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/cdcc1d4d Branch: refs/heads/master Commit: cdcc1d4d41bf16e628b87893814a0ae9f627bcdb Parents: 5a72c37 Author: Bill Havanki Authored: Tue Apr 22 13:44:55 2014 -0400 Committer: Bill Havanki Committed: Mon May 5 09:31:54 2014 -0400 ---------------------------------------------------------------------- conf/templates/generic_logger.properties | 58 ++++++++++++++++++++ conf/templates/monitor_logger.properties | 44 +++++++++++++++ .../org/apache/accumulo/server/Accumulo.java | 39 +++++++++++-- .../server/watcher/MonitorLog4jWatcher.java | 19 ++++++- .../apache/accumulo/server/AccumuloTest.java | 51 +++++++++++++++++ .../server/watcher/MonitorLog4jWatcherTest.java | 50 +++++++++++++++++ 6 files changed, 252 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/conf/templates/generic_logger.properties ---------------------------------------------------------------------- diff --git a/conf/templates/generic_logger.properties b/conf/templates/generic_logger.properties new file mode 100644 index 0000000..06e28df --- /dev/null +++ b/conf/templates/generic_logger.properties @@ -0,0 +1,58 @@ +# 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. + +# Write out everything at the DEBUG level to the debug log +log4j.appender.A2=org.apache.log4j.RollingFileAppender +log4j.appender.A2.File=${org.apache.accumulo.core.dir.log}/${org.apache.accumulo.core.application}_${org.apache.accumulo.core.ip.localhost.hostname}_fromprops.debug.log +log4j.appender.A2.MaxFileSize=1000MB +log4j.appender.A2.MaxBackupIndex=10 +log4j.appender.A2.Threshold=DEBUG +log4j.appender.A2.layout=org.apache.log4j.PatternLayout +log4j.appender.A2.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n + +# Write out INFO and higher to the regular log +log4j.appender.A3=org.apache.log4j.RollingFileAppender +log4j.appender.A3.File=${org.apache.accumulo.core.dir.log}/${org.apache.accumulo.core.application}_${org.apache.accumulo.core.ip.localhost.hostname}_fromprops.log +log4j.appender.A3.MaxFileSize=1000MB +log4j.appender.A3.MaxBackupIndex=10 +log4j.appender.A3.Threshold=INFO +log4j.appender.A3.layout=org.apache.log4j.PatternLayout +log4j.appender.A3.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %m%n + +# Send all logging data to a centralized logger +# If the centralized logger is down, buffer the log events, but drop them if it stays down +log4j.appender.ASYNC=org.apache.accumulo.core.util.AsyncSocketAppender +log4j.appender.ASYNC.RemoteHost=${org.apache.accumulo.core.host.log} +log4j.appender.ASYNC.Port=4560 +log4j.appender.ASYNC.Application=${org.apache.accumulo.core.application}:${org.apache.accumulo.core.ip.localhost.hostname} +log4j.appender.ASYNC.Threshold=WARN + +# Log accumulo events to the debug, normal and remote logs. +log4j.logger.org.apache.accumulo=DEBUG, A2, A3, ASYNC +log4j.additivity.org.apache.accumulo=false + +# change to INFO for authorization events +log4j.logger.org.apache.accumulo.server.security.Auditor=WARN + +log4j.logger.org.apache.accumulo.core.file.rfile.bcfile=INFO + +log4j.logger.org.apache.accumulo.examples.wikisearch=INFO + +log4j.logger.org.mortbay.log=WARN + +log4j.logger.com.yahoo.zookeeper=ERROR + +# Log non-accumulo events to the debug and normal logs +log4j.rootLogger=INFO, A2, A3 http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/conf/templates/monitor_logger.properties ---------------------------------------------------------------------- diff --git a/conf/templates/monitor_logger.properties b/conf/templates/monitor_logger.properties new file mode 100644 index 0000000..4dada2d --- /dev/null +++ b/conf/templates/monitor_logger.properties @@ -0,0 +1,44 @@ +# 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. + +# Write out everything at the DEBUG level to the debug log +log4j.appender.A2=org.apache.log4j.RollingFileAppender +log4j.appender.A2.File=${org.apache.accumulo.core.dir.log}/${org.apache.accumulo.core.application}_${org.apache.accumulo.core.ip.localhost.hostname}_fromprops.debug.log +log4j.appender.A2.MaxFileSize=100MB +log4j.appender.A2.MaxBackupIndex=10 +log4j.appender.A2.Threshold=DEBUG +log4j.appender.A2.layout=org.apache.log4j.PatternLayout +log4j.appender.A2.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %X{application} %m%n + +# Write out INFO and higher to the regular log +log4j.appender.A3=org.apache.log4j.RollingFileAppender +log4j.appender.A3.File=${org.apache.accumulo.core.dir.log}/${org.apache.accumulo.core.application}_${org.apache.accumulo.core.ip.localhost.hostname}_fromprops.log +log4j.appender.A3.MaxFileSize=100MB +log4j.appender.A3.MaxBackupIndex=10 +log4j.appender.A3.Threshold=INFO +log4j.appender.A3.layout=org.apache.log4j.PatternLayout +log4j.appender.A3.layout.ConversionPattern=%d{ISO8601} [%-8c{2}] %-5p: %X{application} %m%n + +# Keep the last few log messages for display to the user +log4j.appender.GUI=org.apache.accumulo.server.monitor.LogService +log4j.appender.GUI.Keep=50 +log4j.appender.GUI.Threshold=WARN + +# Log accumulo messages to debug, normal and GUI +log4j.logger.org.apache.accumulo=DEBUG, A2, A3, GUI +log4j.additivity.org.apache.accumulo=false + +# Log non-accumulo messages to debug, normal logs. +log4j.rootLogger=INFO, A2, A3 http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java index 4e1eb35..68d862d 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java +++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java @@ -110,6 +110,35 @@ public class Accumulo { } } + /** + * Finds the best log4j configuration file. A generic file is used only if an + * application-specific file is not available. An XML file is preferred over + * a properties file, if possible. + * + * @param confDir directory where configuration files should reside + * @param application application name for configuration file name + * @return configuration file name + */ + static String locateLogConfig(String confDir, String application) { + String explicitConfigFile = System.getProperty("log4j.configuration"); + if (explicitConfigFile != null) { + return explicitConfigFile; + } + String[] configFiles = { + String.format("%s/%s_logger.xml", confDir, application), + String.format("%s/%s_logger.properties", confDir, application), + String.format("%s/generic_logger.xml", confDir), + String.format("%s/generic_logger.properties", confDir) + }; + String defaultConfigFile = configFiles[2]; // generic_logger.xml + for (String f : configFiles) { + if (new File(f).exists()) { + return f; + } + } + return defaultConfigFile; + } + public static void init(VolumeManager fs, ServerConfiguration config, String application) throws UnknownHostException { System.setProperty("org.apache.accumulo.core.application", application); @@ -126,11 +155,7 @@ public class Accumulo { System.setProperty("org.apache.accumulo.core.host.log.port", Integer.toString(logPort)); // Use a specific log config, if it exists - String logConfig = String.format("%s/%s_logger.xml", System.getenv("ACCUMULO_CONF_DIR"), application); - if (!new File(logConfig).exists()) { - // otherwise, use the generic config - logConfig = String.format("%s/generic_logger.xml", System.getenv("ACCUMULO_CONF_DIR")); - } + String logConfig = locateLogConfig(System.getenv("ACCUMULO_CONF_DIR"), application); // Turn off messages about not being able to reach the remote logger... we protect against that. LogLog.setQuietMode(true); @@ -140,7 +165,9 @@ public class Accumulo { DOMConfigurator.configureAndWatch(auditConfig, 5000); // Configure logging using information advertised in zookeeper by the monitor - new MonitorLog4jWatcher(config.getInstance().getInstanceID(), logConfig, 5000).start(); + MonitorLog4jWatcher logConfigWatcher = new MonitorLog4jWatcher(config.getInstance().getInstanceID(), logConfig); + logConfigWatcher.setDelay(5000L); + logConfigWatcher.start(); log.info(application + " starting"); log.info("Instance " + config.getInstance().getInstanceID()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java index ac3426e..bc5e99f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java +++ b/server/base/src/main/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcher.java @@ -22,6 +22,7 @@ import org.apache.accumulo.server.zookeeper.ZooReaderWriter; import org.apache.log4j.Appender; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.log4j.PropertyConfigurator; import org.apache.log4j.helpers.FileWatchdog; import org.apache.log4j.xml.DOMConfigurator; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -40,16 +41,24 @@ public class MonitorLog4jWatcher extends FileWatchdog implements Watcher { private static final String PORT_PROPERTY_NAME = "org.apache.accumulo.core.host.log.port"; private final Object lock; + private final boolean usingProperties; private boolean loggingDisabled = false; protected String path; - public MonitorLog4jWatcher(String instance, String filename, int delay) { + public MonitorLog4jWatcher(String instance, String filename) { super(filename); - setDelay(delay); + usingProperties = (filename != null && filename.endsWith(".properties")); this.path = ZooUtil.getRoot(instance) + Constants.ZMONITOR_LOG4J_ADDR; this.lock = new Object(); } + boolean isUsingProperties() { + return usingProperties; + } + String getPath() { + return path; + } + @Override public void run() { try { @@ -85,7 +94,11 @@ public class MonitorLog4jWatcher extends FileWatchdog implements Watcher { private void resetLogger() { // Force a reset on the logger's configuration LogManager.resetConfiguration(); - new DOMConfigurator().doConfigure(filename, LogManager.getLoggerRepository()); + if (usingProperties) { + new PropertyConfigurator().doConfigure(filename, LogManager.getLoggerRepository()); + } else { + new DOMConfigurator().doConfigure(filename, LogManager.getLoggerRepository()); + } } @Override http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java index 9366163..a090b40 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java @@ -16,7 +16,9 @@ */ package org.apache.accumulo.server; +import java.io.File; import java.io.FileNotFoundException; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -24,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.replay; @@ -82,4 +85,52 @@ public class AccumuloTest { assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path)); } + @Test + public void testLocateLogConfig() throws Exception { + File confDir = new File(FileUtils.getTempDirectory(), "AccumuloTest" + System.currentTimeMillis()); + String confDirName = confDir.getAbsolutePath(); + assertTrue("Failed to make test configuration directory", confDir.mkdir()); + try { + File appProps = new File(confDir, "flogger_logger.properties"); + String appPropsName = String.format("%s/flogger_logger.properties", confDirName); + FileUtils.touch(appProps); + File genericXml = new File(confDir, "generic_logger.xml"); + String genericXmlName = String.format("%s/generic_logger.xml", confDirName); + FileUtils.touch(genericXml); + + assertEquals(appPropsName, Accumulo.locateLogConfig(confDirName, "flogger")); + assertEquals(genericXmlName, Accumulo.locateLogConfig(confDirName, "flagger")); + + assertTrue("Failed to delete log configuration file", appProps.delete()); + assertEquals(genericXmlName, Accumulo.locateLogConfig(confDirName, "flogger")); + } finally { + FileUtils.deleteDirectory(confDir); + } + } + @Test + public void testLocateLogConfig_Default() throws Exception { + File confDir = new File(FileUtils.getTempDirectory(), "AccumuloTest" + System.currentTimeMillis()); + String confDirName = confDir.getAbsolutePath(); + assertTrue("Failed to make test configuration directory", confDir.mkdir()); + try { + File genericXml = new File(confDir, "generic_logger.xml"); + String genericXmlName = String.format("%s/generic_logger.xml", confDirName); + + assertEquals(genericXmlName, Accumulo.locateLogConfig(confDirName, "flogger")); + } finally { + FileUtils.deleteDirectory(confDir); + } + } + @Test + public void testLocateLogConfig_Explicit() throws Exception { + File confDir = new File(FileUtils.getTempDirectory(), "AccumuloTest" + System.currentTimeMillis()); + String confDirName = confDir.getAbsolutePath(); + System.setProperty("log4j.configuration", "myconfig.xml"); + try { + assertEquals("myconfig.xml", Accumulo.locateLogConfig(confDirName, "flogger")); + } finally { + FileUtils.deleteDirectory(confDir); + System.clearProperty("log4j.configuration"); + } + } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/cdcc1d4d/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java b/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java new file mode 100644 index 0000000..448e0ac --- /dev/null +++ b/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java @@ -0,0 +1,50 @@ +/* + * 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 org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.zookeeper.ZooUtil; + +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; +import org.easymock.Capture; +import static org.easymock.EasyMock.*; + +public class MonitorLog4jWatcherTest { + private static final String INSTANCE = "instance"; + private static final String FILENAME = "something_logger.xml"; + + private MonitorLog4jWatcher w; + + @Before + public void setUp() throws Exception { + w = new MonitorLog4jWatcher(INSTANCE, FILENAME); + } + + @Test + public void testGetters() { + assertFalse(w.isUsingProperties()); + assertEquals(ZooUtil.getRoot(INSTANCE) + Constants.ZMONITOR_LOG4J_ADDR, w.getPath()); + } + + @Test + public void testPropertyDetection() { + w = new MonitorLog4jWatcher(INSTANCE, FILENAME + ".properties"); + assertTrue(w.isUsingProperties()); + } +}