HADOOP-14944. Add JvmMetrics to KMS.
This commit is contained in:
parent
12c81c67d7
commit
a6370dde90
|
@ -58,6 +58,11 @@ public class JvmMetrics implements MetricsSource {
|
|||
}
|
||||
return impl;
|
||||
}
|
||||
|
||||
synchronized void shutdown() {
|
||||
DefaultMetricsSystem.instance().unregisterSource(JvmMetrics.name());
|
||||
impl = null;
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -81,6 +86,7 @@ public class JvmMetrics implements MetricsSource {
|
|||
final ConcurrentHashMap<String, MetricsInfo[]> gcInfoCache =
|
||||
new ConcurrentHashMap<String, MetricsInfo[]>();
|
||||
|
||||
@VisibleForTesting
|
||||
JvmMetrics(String processName, String sessionId) {
|
||||
this.processName = processName;
|
||||
this.sessionId = sessionId;
|
||||
|
@ -104,6 +110,16 @@ public class JvmMetrics implements MetricsSource {
|
|||
return Singleton.INSTANCE.init(processName, sessionId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown the JvmMetrics singleton. This is not necessary if the JVM itself
|
||||
* is shutdown, but may be necessary for scenarios where JvmMetrics instance
|
||||
* needs to be re-created while the JVM is still around. One such scenario
|
||||
* is unit-testing.
|
||||
*/
|
||||
public static void shutdownSingleton() {
|
||||
Singleton.INSTANCE.shutdown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getMetrics(MetricsCollector collector, boolean all) {
|
||||
MetricsRecordBuilder rb = collector.addRecord(JvmMetrics)
|
||||
|
|
|
@ -63,6 +63,15 @@ public class KMSConfiguration {
|
|||
public static final String KMS_AUDIT_AGGREGATION_WINDOW = CONFIG_PREFIX +
|
||||
"audit.aggregation.window.ms";
|
||||
|
||||
// Process name shown in metrics
|
||||
public static final String METRICS_PROCESS_NAME_KEY =
|
||||
CONFIG_PREFIX + "metrics.process.name";
|
||||
public static final String METRICS_PROCESS_NAME_DEFAULT = "KMS";
|
||||
|
||||
// Session id for metrics
|
||||
public static final String METRICS_SESSION_ID_KEY =
|
||||
CONFIG_PREFIX + "metrics.session.id";
|
||||
|
||||
// KMS Audit logger classes to use
|
||||
public static final String KMS_AUDIT_LOGGER_KEY = CONFIG_PREFIX +
|
||||
"audit.logger";
|
||||
|
|
|
@ -28,8 +28,11 @@ import org.apache.hadoop.crypto.key.KeyProvider;
|
|||
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
|
||||
import org.apache.hadoop.crypto.key.KeyProviderFactory;
|
||||
import org.apache.hadoop.http.HttpServer2;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.metrics2.source.JvmMetrics;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.util.JvmPauseMonitor;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
import org.apache.log4j.PropertyConfigurator;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -44,6 +47,10 @@ import java.io.IOException;
|
|||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
|
||||
import static org.apache.hadoop.crypto.key.kms.server.KMSConfiguration.METRICS_PROCESS_NAME_DEFAULT;
|
||||
import static org.apache.hadoop.crypto.key.kms.server.KMSConfiguration.METRICS_PROCESS_NAME_KEY;
|
||||
import static org.apache.hadoop.crypto.key.kms.server.KMSConfiguration.METRICS_SESSION_ID_KEY;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class KMSWebApp implements ServletContextListener {
|
||||
|
||||
|
@ -78,6 +85,9 @@ public class KMSWebApp implements ServletContextListener {
|
|||
private static Meter decryptEEKCallsMeter;
|
||||
private static Meter generateEEKCallsMeter;
|
||||
private static Meter invalidCallsMeter;
|
||||
private static String processName;
|
||||
private static String sessionId;
|
||||
private static JvmPauseMonitor pauseMonitor;
|
||||
private static KMSAudit kmsAudit;
|
||||
private static KeyProviderCryptoExtension keyProviderCryptoExtension;
|
||||
|
||||
|
@ -149,6 +159,16 @@ public class KMSWebApp implements ServletContextListener {
|
|||
unauthenticatedCallsMeter = metricRegistry.register(
|
||||
UNAUTHENTICATED_CALLS_METER, new Meter());
|
||||
|
||||
processName =
|
||||
kmsConf.get(METRICS_PROCESS_NAME_KEY, METRICS_PROCESS_NAME_DEFAULT);
|
||||
sessionId = kmsConf.get(METRICS_SESSION_ID_KEY);
|
||||
pauseMonitor = new JvmPauseMonitor();
|
||||
pauseMonitor.init(kmsConf);
|
||||
DefaultMetricsSystem.initialize(processName);
|
||||
final JvmMetrics jm = JvmMetrics.initSingleton(processName, sessionId);
|
||||
jm.setPauseMonitor(pauseMonitor);
|
||||
pauseMonitor.start();
|
||||
|
||||
kmsAudit = new KMSAudit(kmsConf);
|
||||
|
||||
// this is required for the the JMXJsonServlet to work properly.
|
||||
|
@ -222,6 +242,9 @@ public class KMSWebApp implements ServletContextListener {
|
|||
}
|
||||
kmsAudit.shutdown();
|
||||
kmsAcls.stopReloader();
|
||||
pauseMonitor.stop();
|
||||
JvmMetrics.shutdownSingleton();
|
||||
DefaultMetricsSystem.shutdown();
|
||||
jmxReporter.stop();
|
||||
jmxReporter.close();
|
||||
metricRegistry = null;
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.junit.rules.Timeout;
|
|||
import org.slf4j.event.Level;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import javax.security.auth.login.AppConfigurationEntry;
|
||||
|
||||
|
@ -62,6 +61,7 @@ import java.io.DataInputStream;
|
|||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.Writer;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -82,6 +82,8 @@ import java.util.Set;
|
|||
import java.util.UUID;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestKMS {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class);
|
||||
|
||||
|
@ -2380,4 +2382,38 @@ public class TestKMS {
|
|||
});
|
||||
}
|
||||
|
||||
/*
|
||||
* Test the jmx page can return, and contains the basic JvmMetrics. Only
|
||||
* testing in simple mode since the page content is the same, kerberized
|
||||
* or not.
|
||||
*/
|
||||
@Test
|
||||
public void testKMSJMX() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
final File confDir = getTestDir();
|
||||
conf = createBaseKMSConf(confDir, conf);
|
||||
final String processName = "testkmsjmx";
|
||||
conf.set(KMSConfiguration.METRICS_PROCESS_NAME_KEY, processName);
|
||||
writeConf(confDir, conf);
|
||||
|
||||
runServer(null, null, confDir, new KMSCallable<Void>() {
|
||||
@Override
|
||||
public Void call() throws Exception {
|
||||
final URL jmxUrl = new URL(
|
||||
getKMSUrl() + "/jmx?user.name=whatever&qry=Hadoop:service="
|
||||
+ processName + ",name=JvmMetrics");
|
||||
LOG.info("Requesting jmx from " + jmxUrl);
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
final InputStream in = jmxUrl.openConnection().getInputStream();
|
||||
final byte[] buffer = new byte[64 * 1024];
|
||||
int len;
|
||||
while ((len = in.read(buffer)) > 0) {
|
||||
sb.append(new String(buffer, 0, len));
|
||||
}
|
||||
LOG.info("jmx returned: " + sb.toString());
|
||||
assertTrue(sb.toString().contains("JvmMetrics"));
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue