HADOOP-18648. Avoid loading kms log4j properties dynamically by KMSWebServer (#5441)

This commit is contained in:
Viraj Jasani 2023-03-01 16:02:07 -08:00 committed by GitHub
parent 162288bc0a
commit e1ca466bdb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 16 additions and 27 deletions

View File

@ -20,7 +20,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.PropertyConfigurator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -104,8 +104,6 @@ public class KMSConfiguration {
public static final boolean KEY_AUTHORIZATION_ENABLE_DEFAULT = true;
private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
static {
Configuration.addDefaultResource(KMS_DEFAULT_XML);
Configuration.addDefaultResource(KMS_SITE_XML);
@ -163,31 +161,20 @@ public static boolean isACLsFileNewer(long time) {
return newer;
}
public static void initLogging() {
String confDir = System.getProperty(KMS_CONFIG_DIR);
if (confDir == null) {
throw new RuntimeException("System property '" +
KMSConfiguration.KMS_CONFIG_DIR + "' not defined");
/**
* Validate whether "kms.config.dir" and "log4j.configuration" are defined in the System
* properties. If not, abort the KMS WebServer.
*/
public static void validateSystemProps() {
if (System.getProperty(KMS_CONFIG_DIR) == null) {
String errorMsg = "System property '" + KMS_CONFIG_DIR + "' not defined";
System.err.println("Aborting KMSWebServer because " + errorMsg);
throw new RuntimeException(errorMsg);
}
if (System.getProperty("log4j.configuration") == null) {
System.setProperty("log4j.defaultInitOverride", "true");
boolean fromClasspath = true;
File log4jConf = new File(confDir, LOG4J_PROPERTIES).getAbsoluteFile();
if (log4jConf.exists()) {
PropertyConfigurator.configureAndWatch(log4jConf.getPath(), 1000);
fromClasspath = false;
} else {
ClassLoader cl = Thread.currentThread().getContextClassLoader();
URL log4jUrl = cl.getResource(LOG4J_PROPERTIES);
if (log4jUrl != null) {
PropertyConfigurator.configure(log4jUrl);
}
}
LOG.debug("KMS log starting");
if (fromClasspath) {
LOG.warn("Log4j configuration file '{}' not found", LOG4J_PROPERTIES);
LOG.warn("Logging with INFO level to standard output");
}
String errorMsg = "System property 'log4j.configuration' not defined";
System.err.println("Aborting KMSWebServer because " + errorMsg);
throw new RuntimeException(errorMsg);
}
}
}

View File

@ -185,7 +185,7 @@ public URL getKMSUrl() {
}
public static void main(String[] args) throws Exception {
KMSConfiguration.initLogging();
KMSConfiguration.validateSystemProps();
StringUtils.startupShutdownMessage(KMSWebServer.class, args, LOG);
Configuration conf = KMSConfiguration.getKMSConf();
Configuration sslConf = SSLFactory.readSSLConfiguration(conf, SSLFactory.Mode.SERVER);

View File

@ -49,6 +49,8 @@ function hadoop_subcommand_kms
"-Dkms.config.dir=${HADOOP_CONF_DIR}"
hadoop_add_param HADOOP_OPTS "-Dkms.log.dir=" \
"-Dkms.log.dir=${HADOOP_LOG_DIR}"
hadoop_add_param HADOOP_OPTS "-Dlog4j.configuration=" \
"-Dlog4j.configuration=file:${HADOOP_CONF_DIR}/kms-log4j.properties"
if [[ "${HADOOP_DAEMON_MODE}" == "default" ]] ||
[[ "${HADOOP_DAEMON_MODE}" == "start" ]]; then