You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2018/09/17 19:54:02 UTC
[1/6] storm git commit: STORM-3197: Make StormMetricsRegistry
non-static
Repository: storm
Updated Branches:
refs/heads/master 37d22b8df -> eaed3cbf3
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeterNames.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeterNames.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeterNames.java
new file mode 100644
index 0000000..61055ba
--- /dev/null
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeterNames.java
@@ -0,0 +1,60 @@
+
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.storm.daemon.logviewer.utils;
+
+import org.apache.storm.metric.StormMetricsRegistry;
+
+public class ExceptionMeterNames {
+
+ //Operation level IO Exceptions
+ public static final String NUM_FILE_OPEN_EXCEPTIONS = "logviewer:num-file-open-exceptions";
+ public static final String NUM_FILE_READ_EXCEPTIONS = "logviewer:num-file-read-exceptions";
+ public static final String NUM_FILE_REMOVAL_EXCEPTIONS = "logviewer:num-file-removal-exceptions";
+ public static final String NUM_FILE_DOWNLOAD_EXCEPTIONS = "logviewer:num-file-download-exceptions";
+ public static final String NUM_SET_PERMISSION_EXCEPTIONS = "logviewer:num-set-permission-exceptions";
+
+ //Routine level
+ public static final String NUM_CLEANUP_EXCEPTIONS = "logviewer:num-other-cleanup-exceptions";
+ public static final String NUM_READ_LOG_EXCEPTIONS = "logviewer:num-read-log-exceptions";
+ public static final String NUM_READ_DAEMON_LOG_EXCEPTIONS = "logviewer:num-read-daemon-log-exceptions";
+ public static final String NUM_LIST_LOG_EXCEPTIONS = "logviewer:num-search-log-exceptions";
+ public static final String NUM_LIST_DUMP_EXCEPTIONS = "logviewer:num-list-dump-files-exceptions";
+ public static final String NUM_DOWNLOAD_DUMP_EXCEPTIONS = "logviewer:num-download-dump-exceptions";
+ public static final String NUM_DOWNLOAD_LOG_EXCEPTIONS = "logviewer:num-download-log-exceptions";
+ public static final String NUM_DOWNLOAD_DAEMON_LOG_EXCEPTIONS = "logviewer:num-download-daemon-log-exceptions";
+ public static final String NUM_SEARCH_EXCEPTIONS = "logviewer:num-search-exceptions";
+
+ /**
+ * It may be helpful to register these meters up front, so they are output even if their values are zero.
+ * @param registry The metrics registry.
+ */
+ public static void registerMeters(StormMetricsRegistry registry) {
+
+ registry.registerMeter(NUM_FILE_OPEN_EXCEPTIONS);
+ registry.registerMeter(NUM_FILE_READ_EXCEPTIONS);
+ registry.registerMeter(NUM_FILE_REMOVAL_EXCEPTIONS);
+ registry.registerMeter(NUM_FILE_DOWNLOAD_EXCEPTIONS);
+ registry.registerMeter(NUM_SET_PERMISSION_EXCEPTIONS);
+ registry.registerMeter(NUM_CLEANUP_EXCEPTIONS);
+ registry.registerMeter(NUM_READ_LOG_EXCEPTIONS);
+ registry.registerMeter(NUM_READ_DAEMON_LOG_EXCEPTIONS);
+ registry.registerMeter(NUM_LIST_LOG_EXCEPTIONS);
+ registry.registerMeter(NUM_LIST_DUMP_EXCEPTIONS);
+ registry.registerMeter(NUM_DOWNLOAD_DUMP_EXCEPTIONS);
+ registry.registerMeter(NUM_DOWNLOAD_LOG_EXCEPTIONS);
+ registry.registerMeter(NUM_DOWNLOAD_DAEMON_LOG_EXCEPTIONS);
+ registry.registerMeter(NUM_SEARCH_EXCEPTIONS);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeters.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeters.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeters.java
deleted file mode 100644
index 81aa222..0000000
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/ExceptionMeters.java
+++ /dev/null
@@ -1,66 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.storm.daemon.logviewer.utils;
-
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.Metric;
-import com.codahale.metrics.MetricSet;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public enum ExceptionMeters {
- //Operation level IO Exceptions
- NUM_FILE_OPEN_EXCEPTIONS("logviewer:num-file-open-exceptions"),
- NUM_FILE_READ_EXCEPTIONS("logviewer:num-file-read-exceptions"),
- NUM_FILE_REMOVAL_EXCEPTIONS("logviewer:num-file-removal-exceptions"),
- NUM_FILE_DOWNLOAD_EXCEPTIONS("logviewer:num-file-download-exceptions"),
- NUM_SET_PERMISSION_EXCEPTIONS("logviewer:num-set-permission-exceptions"),
-
- //Routine level
- NUM_CLEANUP_EXCEPTIONS("logviewer:num-other-cleanup-exceptions"),
- NUM_READ_LOG_EXCEPTIONS("logviewer:num-read-log-exceptions"),
- NUM_READ_DAEMON_LOG_EXCEPTIONS("logviewer:num-read-daemon-log-exceptions"),
- NUM_LIST_LOG_EXCEPTIONS("logviewer:num-search-log-exceptions"),
- NUM_LIST_DUMP_EXCEPTIONS("logviewer:num-list-dump-files-exceptions"),
- NUM_DOWNLOAD_DUMP_EXCEPTIONS("logviewer:num-download-dump-exceptions"),
- NUM_DOWNLOAD_LOG_EXCEPTIONS("logviewer:num-download-log-exceptions"),
- NUM_DOWNLOAD_DAEMON_LOG_EXCEPTIONS("logviewer:num-download-daemon-log-exceptions"),
- NUM_SEARCH_EXCEPTIONS("logviewer:num-search-exceptions");
-
- private static final Map<String, Metric> metrics = new HashMap<>();
-
- static {
- for (ExceptionMeters e : ExceptionMeters.values()) {
- metrics.put(e.name, e.meter);
- }
- }
-
- private final String name;
- private final Meter meter;
-
- public static Map<String, Metric> getMetrics() {
- return metrics;
- }
-
- ExceptionMeters(String name) {
- this.name = name;
- meter = new Meter();
- }
-
- public void mark() {
- this.meter.mark();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogCleaner.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogCleaner.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogCleaner.java
index ae8aff6..035fe3b 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogCleaner.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogCleaner.java
@@ -28,6 +28,7 @@ import static org.apache.storm.DaemonConfig.LOGVIEWER_MAX_PER_WORKER_LOGS_SIZE_M
import static org.apache.storm.DaemonConfig.LOGVIEWER_MAX_SUM_WORKER_LOGS_SIZE_MB;
import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting;
@@ -66,10 +67,13 @@ import org.slf4j.LoggerFactory;
*/
public class LogCleaner implements Runnable, Closeable {
private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class);
- private static final Timer cleanupRoutineDuration = StormMetricsRegistry.registerTimer("logviewer:cleanup-routine-duration-ms");
- private static final Histogram numFilesCleanedUp = StormMetricsRegistry.registerHistogram("logviewer:num-files-cleaned-up");
- private static final Histogram diskSpaceFreed = StormMetricsRegistry.registerHistogram("logviewer:disk-space-freed-in-bytes");
+ private final Timer cleanupRoutineDuration;
+ private final Histogram numFilesCleanedUp;
+ private final Histogram diskSpaceFreed;
+ private final Meter numFileRemovalExceptions;
+ private final Meter numCleanupExceptions;
+
private final Map<String, Object> stormConf;
private final Integer intervalSecs;
private final File logRootDir;
@@ -87,9 +91,10 @@ public class LogCleaner implements Runnable, Closeable {
* @param workerLogs {@link WorkerLogs} instance
* @param directoryCleaner {@link DirectoryCleaner} instance
* @param logRootDir root log directory
+ * @param metricsRegistry The logviewer metrics registry
*/
public LogCleaner(Map<String, Object> stormConf, WorkerLogs workerLogs, DirectoryCleaner directoryCleaner,
- File logRootDir) {
+ File logRootDir, StormMetricsRegistry metricsRegistry) {
this.stormConf = stormConf;
this.intervalSecs = ObjectReader.getInt(stormConf.get(LOGVIEWER_CLEANUP_INTERVAL_SECS), null);
this.logRootDir = logRootDir;
@@ -101,9 +106,14 @@ public class LogCleaner implements Runnable, Closeable {
maxPerWorkerLogsSizeMb = Math.min(maxPerWorkerLogsSizeMb, (long) (maxSumWorkerLogsSizeMb * 0.5));
LOG.info("configured max total size of worker logs: {} MB, max total size of worker logs per directory: {} MB",
- maxSumWorkerLogsSizeMb, maxPerWorkerLogsSizeMb);
+ maxSumWorkerLogsSizeMb, maxPerWorkerLogsSizeMb);
//Switch to CachedGauge if this starts to hurt performance
- StormMetricsRegistry.registerGauge("logviewer:worker-log-dir-size", () -> FileUtils.sizeOf(logRootDir));
+ metricsRegistry.registerGauge("logviewer:worker-log-dir-size", () -> FileUtils.sizeOf(logRootDir));
+ this.cleanupRoutineDuration = metricsRegistry.registerTimer("logviewer:cleanup-routine-duration-ms");
+ this.numFilesCleanedUp = metricsRegistry.registerHistogram("logviewer:num-files-cleaned-up");
+ this.diskSpaceFreed = metricsRegistry.registerHistogram("logviewer:disk-space-freed-in-bytes");
+ this.numFileRemovalExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_REMOVAL_EXCEPTIONS);
+ this.numCleanupExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_CLEANUP_EXCEPTIONS);
}
/**
@@ -164,7 +174,7 @@ public class LogCleaner implements Runnable, Closeable {
numFilesCleaned++;
diskSpaceCleaned += sizeInBytes;
} catch (Exception ex) {
- ExceptionMeters.NUM_FILE_REMOVAL_EXCEPTIONS.mark();
+ numFileRemovalExceptions.mark();
LOG.error(ex.getMessage(), ex);
}
}
@@ -176,7 +186,7 @@ public class LogCleaner implements Runnable, Closeable {
numFilesCleaned += globalLogCleanupMeta.deletedFiles;
diskSpaceCleaned += globalLogCleanupMeta.deletedSize;
} catch (Exception ex) {
- ExceptionMeters.NUM_CLEANUP_EXCEPTIONS.mark();
+ numCleanupExceptions.mark();
LOG.error("Exception while cleaning up old log.", ex);
}
numFilesCleanedUp.update(numFilesCleaned);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
index 75cb29f..35cba34 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
@@ -19,6 +19,7 @@
package org.apache.storm.daemon.logviewer.utils;
import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
import java.io.File;
import java.io.IOException;
@@ -30,8 +31,8 @@ import org.apache.storm.metric.StormMetricsRegistry;
public class LogFileDownloader {
- private static final Histogram fileDownloadSizeDistMB = StormMetricsRegistry.registerHistogram("logviewer:download-file-size-rounded-MB");
-
+ private final Histogram fileDownloadSizeDistMb;
+ private final Meter numFileDownloadExceptions;
private final String logRoot;
private final String daemonLogRoot;
private final ResourceAuthorizer resourceAuthorizer;
@@ -42,11 +43,15 @@ public class LogFileDownloader {
* @param logRoot root worker log directory
* @param daemonLogRoot root daemon log directory
* @param resourceAuthorizer {@link ResourceAuthorizer}
+ * @param metricsRegistry The logviewer metrics registry
*/
- public LogFileDownloader(String logRoot, String daemonLogRoot, ResourceAuthorizer resourceAuthorizer) {
+ public LogFileDownloader(String logRoot, String daemonLogRoot, ResourceAuthorizer resourceAuthorizer,
+ StormMetricsRegistry metricsRegistry) {
this.logRoot = logRoot;
this.daemonLogRoot = daemonLogRoot;
this.resourceAuthorizer = resourceAuthorizer;
+ this.fileDownloadSizeDistMb = metricsRegistry.registerHistogram("logviewer:download-file-size-rounded-MB");
+ this.numFileDownloadExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_DOWNLOAD_EXCEPTIONS);
}
/**
@@ -62,8 +67,8 @@ public class LogFileDownloader {
File file = new File(rootDir, fileName).getCanonicalFile();
if (file.exists()) {
if (isDaemon || resourceAuthorizer.isUserAllowedToAccessFile(user, fileName)) {
- fileDownloadSizeDistMB.update(Math.round((double) file.length() / FileUtils.ONE_MB));
- return LogviewerResponseBuilder.buildDownloadFile(file);
+ fileDownloadSizeDistMb.update(Math.round((double) file.length() / FileUtils.ONE_MB));
+ return LogviewerResponseBuilder.buildDownloadFile(file, numFileDownloadExceptions);
} else {
return LogviewerResponseBuilder.buildResponseUnauthorizedUser(user);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogviewerResponseBuilder.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogviewerResponseBuilder.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogviewerResponseBuilder.java
index 4c8a191..406f91f 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogviewerResponseBuilder.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogviewerResponseBuilder.java
@@ -24,6 +24,7 @@ import static javax.ws.rs.core.Response.Status.FORBIDDEN;
import static javax.ws.rs.core.Response.Status.OK;
import static org.apache.commons.lang.StringEscapeUtils.escapeHtml;
+import com.codahale.metrics.Meter;
import com.google.common.io.ByteStreams;
import java.io.BufferedOutputStream;
@@ -75,7 +76,7 @@ public class LogviewerResponseBuilder {
*
* @param file file to download
*/
- public static Response buildDownloadFile(File file) throws IOException {
+ public static Response buildDownloadFile(File file, Meter numFileDownloadExceptions) throws IOException {
try {
// do not close this InputStream in method: it will be used from jetty server
InputStream is = new FileInputStream(file);
@@ -85,7 +86,7 @@ public class LogviewerResponseBuilder {
.header("Content-Disposition", "attachment; filename=\"" + file.getName() + "\"")
.build();
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_DOWNLOAD_EXCEPTIONS.mark();
+ numFileDownloadExceptions.mark();
throw e;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/WorkerLogs.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/WorkerLogs.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/WorkerLogs.java
index d566e3d..8d9934c 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/WorkerLogs.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/WorkerLogs.java
@@ -23,6 +23,7 @@ import static java.util.stream.Collectors.toMap;
import static org.apache.storm.Config.SUPERVISOR_RUN_WORKER_AS_USER;
import static org.apache.storm.Config.TOPOLOGY_SUBMITTER_USER;
+import com.codahale.metrics.Meter;
import com.google.common.collect.Lists;
import java.io.File;
@@ -42,6 +43,7 @@ import java.util.stream.Stream;
import org.apache.storm.daemon.supervisor.ClientSupervisorUtils;
import org.apache.storm.daemon.supervisor.SupervisorUtils;
import org.apache.storm.daemon.utils.PathUtil;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.ObjectReader;
import org.apache.storm.utils.Time;
import org.apache.storm.utils.Utils;
@@ -57,18 +59,25 @@ public class WorkerLogs {
private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class);
public static final String WORKER_YAML = "worker.yaml";
+
+ private final Meter numSetPermissionsExceptions;
+
private final Map<String, Object> stormConf;
private final File logRootDir;
+ private final DirectoryCleaner directoryCleaner;
/**
* Constructor.
*
* @param stormConf storm configuration
* @param logRootDir the log root directory
+ * @param metricsRegistry The logviewer metrics registry
*/
- public WorkerLogs(Map<String, Object> stormConf, File logRootDir) {
+ public WorkerLogs(Map<String, Object> stormConf, File logRootDir, StormMetricsRegistry metricsRegistry) {
this.stormConf = stormConf;
this.logRootDir = logRootDir;
+ this.numSetPermissionsExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_SET_PERMISSION_EXCEPTIONS);
+ this.directoryCleaner = new DirectoryCleaner(metricsRegistry);
}
/**
@@ -92,7 +101,7 @@ public class WorkerLogs {
Lists.newArrayList("blob", file.getCanonicalPath()), null,
"setup group read permissions for file: " + fileName);
} catch (IOException e) {
- ExceptionMeters.NUM_SET_PERMISSION_EXCEPTIONS.mark();
+ numSetPermissionsExceptions.mark();
throw e;
}
}
@@ -106,7 +115,7 @@ public class WorkerLogs {
Set<File> topoDirFiles = getAllWorkerDirs();
if (topoDirFiles != null) {
for (File portDir : topoDirFiles) {
- files.addAll(DirectoryCleaner.getFilesForDir(portDir));
+ files.addAll(directoryCleaner.getFilesForDir(portDir));
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerApplication.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerApplication.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerApplication.java
index 9af8c33..6346d2d 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerApplication.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerApplication.java
@@ -39,6 +39,7 @@ import org.apache.storm.daemon.logviewer.handler.LogviewerLogSearchHandler;
import org.apache.storm.daemon.logviewer.handler.LogviewerProfileHandler;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.security.auth.IHttpCredentialsPlugin;
import org.apache.storm.security.auth.ServerAuthUtils;
import org.apache.storm.utils.ConfigUtils;
@@ -47,7 +48,8 @@ import org.apache.storm.utils.ObjectReader;
@ApplicationPath("")
public class LogviewerApplication extends Application {
private static Map<String, Object> stormConf;
- private final Set<Object> singletons = new HashSet<Object>();
+ private static StormMetricsRegistry metricsRegistry;
+ private final Set<Object> singletons = new HashSet<>();
/**
* Constructor.
@@ -57,17 +59,19 @@ public class LogviewerApplication extends Application {
String daemonLogRoot = logRootDir(ObjectReader.getString(stormConf.get(LOGVIEWER_APPENDER_NAME)));
ResourceAuthorizer resourceAuthorizer = new ResourceAuthorizer(stormConf);
- WorkerLogs workerLogs = new WorkerLogs(stormConf, new File(logRoot));
+ WorkerLogs workerLogs = new WorkerLogs(stormConf, new File(logRoot), metricsRegistry);
- LogviewerLogPageHandler logviewer = new LogviewerLogPageHandler(logRoot, daemonLogRoot, workerLogs, resourceAuthorizer);
- LogviewerProfileHandler profileHandler = new LogviewerProfileHandler(logRoot, resourceAuthorizer);
+ LogviewerLogPageHandler logviewer = new LogviewerLogPageHandler(logRoot, daemonLogRoot, workerLogs, resourceAuthorizer,
+ metricsRegistry);
+ LogviewerProfileHandler profileHandler = new LogviewerProfileHandler(logRoot, resourceAuthorizer, metricsRegistry);
LogviewerLogDownloadHandler logDownloadHandler = new LogviewerLogDownloadHandler(logRoot, daemonLogRoot,
- workerLogs, resourceAuthorizer);
+ workerLogs, resourceAuthorizer, metricsRegistry);
LogviewerLogSearchHandler logSearchHandler = new LogviewerLogSearchHandler(stormConf, logRoot, daemonLogRoot,
- resourceAuthorizer);
+ resourceAuthorizer, metricsRegistry);
IHttpCredentialsPlugin httpCredsHandler = ServerAuthUtils.getUiHttpCredentialsPlugin(stormConf);
- singletons.add(new LogviewerResource(logviewer, profileHandler, logDownloadHandler, logSearchHandler, httpCredsHandler));
+ singletons.add(new LogviewerResource(logviewer, profileHandler, logDownloadHandler, logSearchHandler,
+ httpCredsHandler, metricsRegistry));
singletons.add(new AuthorizationExceptionMapper());
}
@@ -80,9 +84,11 @@ public class LogviewerApplication extends Application {
* Spot to inject storm configuration before initializing LogviewerApplication instance.
*
* @param stormConf storm configuration
+ * @param metricRegistry The metrics registry
*/
- public static void setup(Map<String, Object> stormConf) {
+ public static void setup(Map<String, Object> stormConf, StormMetricsRegistry metricRegistry) {
LogviewerApplication.stormConf = stormConf;
+ LogviewerApplication.metricsRegistry = metricRegistry;
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java
index 08881be..3afde07 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java
@@ -39,7 +39,7 @@ import org.apache.storm.daemon.logviewer.handler.LogviewerLogDownloadHandler;
import org.apache.storm.daemon.logviewer.handler.LogviewerLogPageHandler;
import org.apache.storm.daemon.logviewer.handler.LogviewerLogSearchHandler;
import org.apache.storm.daemon.logviewer.handler.LogviewerProfileHandler;
-import org.apache.storm.daemon.logviewer.utils.ExceptionMeters;
+import org.apache.storm.daemon.logviewer.utils.ExceptionMeterNames;
import org.apache.storm.daemon.ui.InvalidRequestException;
import org.apache.storm.daemon.ui.UIHelpers;
import org.apache.storm.daemon.ui.resources.StormApiResource;
@@ -55,22 +55,24 @@ import org.slf4j.LoggerFactory;
public class LogviewerResource {
private static final Logger LOG = LoggerFactory.getLogger(LogviewerResource.class);
- private static final Meter meterLogPageHttpRequests = StormMetricsRegistry.registerMeter("logviewer:num-log-page-http-requests");
- private static final Meter meterDaemonLogPageHttpRequests = StormMetricsRegistry.registerMeter(
- "logviewer:num-daemonlog-page-http-requests");
- private static final Meter meterDownloadLogFileHttpRequests = StormMetricsRegistry.registerMeter(
- "logviewer:num-download-log-file-http-requests");
- private static final Meter meterDownloadLogDaemonFileHttpRequests = StormMetricsRegistry.registerMeter(
- "logviewer:num-download-log-daemon-file-http-requests");
- private static final Meter meterListLogsHttpRequests = StormMetricsRegistry.registerMeter("logviewer:num-list-logs-http-requests");
-
- private static final Meter numSearchLogRequests = StormMetricsRegistry.registerMeter("logviewer:num-search-logs-requests");
- private static final Meter numDeepSearchArchived = StormMetricsRegistry.registerMeter(
- "logviewer:num-deep-search-requests-with-archived");
- private static final Meter numDeepSearchNonArchived = StormMetricsRegistry.registerMeter(
- "logviewer:num-deep-search-requests-without-archived");
- private static final Timer searchLogRequestDuration = StormMetricsRegistry.registerTimer("logviewer:search-requests-duration-ms");
- private static final Timer deepSearchRequestDuration = StormMetricsRegistry.registerTimer("logviewer:deep-search-request-duration-ms");
+ private final Meter meterLogPageHttpRequests;
+ private final Meter meterDaemonLogPageHttpRequests;
+ private final Meter meterDownloadLogFileHttpRequests;
+ private final Meter meterDownloadLogDaemonFileHttpRequests;
+ private final Meter meterListLogsHttpRequests;
+ private final Meter numSearchLogRequests;
+ private final Meter numDeepSearchArchived;
+ private final Meter numDeepSearchNonArchived;
+ private final Meter numReadLogExceptions;
+ private final Meter numReadDaemonLogExceptions;
+ private final Meter numListLogExceptions;
+ private final Meter numListDumpExceptions;
+ private final Meter numDownloadDumpExceptions;
+ private final Meter numDownloadLogExceptions;
+ private final Meter numDownloadDaemonLogExceptions;
+ private final Meter numSearchExceptions;
+ private final Timer searchLogRequestDuration;
+ private final Timer deepSearchRequestDuration;
private final LogviewerLogPageHandler logviewer;
private final LogviewerProfileHandler profileHandler;
@@ -86,10 +88,32 @@ public class LogviewerResource {
* @param logDownloadHandler {@link LogviewerLogDownloadHandler}
* @param logSearchHandler {@link LogviewerLogSearchHandler}
* @param httpCredsHandler {@link IHttpCredentialsPlugin}
+ * @param metricsRegistry The metrics registry
*/
public LogviewerResource(LogviewerLogPageHandler logviewerParam, LogviewerProfileHandler profileHandler,
LogviewerLogDownloadHandler logDownloadHandler, LogviewerLogSearchHandler logSearchHandler,
- IHttpCredentialsPlugin httpCredsHandler) {
+ IHttpCredentialsPlugin httpCredsHandler, StormMetricsRegistry metricsRegistry) {
+ this.meterLogPageHttpRequests = metricsRegistry.registerMeter("logviewer:num-log-page-http-requests");
+ this.meterDaemonLogPageHttpRequests = metricsRegistry.registerMeter(
+ "logviewer:num-daemonlog-page-http-requests");
+ this.meterDownloadLogFileHttpRequests = metricsRegistry.registerMeter(
+ "logviewer:num-download-log-file-http-requests");
+ this.meterDownloadLogDaemonFileHttpRequests = metricsRegistry.registerMeter(
+ "logviewer:num-download-log-daemon-file-http-requests");
+ this.meterListLogsHttpRequests = metricsRegistry.registerMeter("logviewer:num-list-logs-http-requests");
+ this.numSearchLogRequests = metricsRegistry.registerMeter("logviewer:num-search-logs-requests");
+ this.numDeepSearchArchived = metricsRegistry.registerMeter("logviewer:num-deep-search-requests-with-archived");
+ this.numDeepSearchNonArchived = metricsRegistry.registerMeter("logviewer:num-deep-search-requests-without-archived");
+ this.numReadLogExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_READ_LOG_EXCEPTIONS);
+ this.numReadDaemonLogExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_READ_DAEMON_LOG_EXCEPTIONS);
+ this.numListLogExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_LIST_LOG_EXCEPTIONS);
+ this.numListDumpExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_LIST_DUMP_EXCEPTIONS);
+ this.numDownloadDumpExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_DOWNLOAD_DUMP_EXCEPTIONS);
+ this.numDownloadLogExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_DOWNLOAD_LOG_EXCEPTIONS);
+ this.numDownloadDaemonLogExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_DOWNLOAD_DAEMON_LOG_EXCEPTIONS);
+ this.numSearchExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_SEARCH_EXCEPTIONS);
+ this.searchLogRequestDuration = metricsRegistry.registerTimer("logviewer:search-requests-duration-ms");
+ this.deepSearchRequestDuration = metricsRegistry.registerTimer("logviewer:deep-search-request-duration-ms");
this.logviewer = logviewerParam;
this.profileHandler = profileHandler;
this.logDownloadHandler = logDownloadHandler;
@@ -116,7 +140,7 @@ public class LogviewerResource {
LOG.error(e.getMessage(), e);
return Response.status(400).entity(e.getMessage()).build();
} catch (IOException e) {
- ExceptionMeters.NUM_READ_LOG_EXCEPTIONS.mark();
+ numReadLogExceptions.mark();
throw e;
}
}
@@ -140,7 +164,7 @@ public class LogviewerResource {
LOG.error(e.getMessage(), e);
return Response.status(400).entity(e.getMessage()).build();
} catch (IOException e) {
- ExceptionMeters.NUM_READ_DAEMON_LOG_EXCEPTIONS.mark();
+ numReadDaemonLogExceptions.mark();
throw e;
}
}
@@ -177,7 +201,7 @@ public class LogviewerResource {
try {
return logviewer.listLogFiles(user, portStr != null ? Integer.parseInt(portStr) : null, topologyId, callback, origin);
} catch (IOException e) {
- ExceptionMeters.NUM_LIST_LOG_EXCEPTIONS.mark();
+ numListLogExceptions.mark();
throw e;
}
}
@@ -193,7 +217,7 @@ public class LogviewerResource {
try {
return profileHandler.listDumpFiles(topologyId, hostPort, user);
} catch (IOException e) {
- ExceptionMeters.NUM_LIST_DUMP_EXCEPTIONS.mark();
+ numListDumpExceptions.mark();
throw e;
}
}
@@ -209,7 +233,7 @@ public class LogviewerResource {
try {
return profileHandler.downloadDumpFile(topologyId, hostPort, fileName, user);
} catch (IOException e) {
- ExceptionMeters.NUM_DOWNLOAD_DUMP_EXCEPTIONS.mark();
+ numDownloadDumpExceptions.mark();
throw e;
}
}
@@ -228,7 +252,7 @@ public class LogviewerResource {
try {
return logDownloadHandler.downloadLogFile(decodedFileName, user);
} catch (IOException e) {
- ExceptionMeters.NUM_DOWNLOAD_LOG_EXCEPTIONS.mark();
+ numDownloadLogExceptions.mark();
throw e;
}
}
@@ -247,7 +271,7 @@ public class LogviewerResource {
try {
return logDownloadHandler.downloadDaemonLogFile(decodedFileName, user);
} catch (IOException e) {
- ExceptionMeters.NUM_DOWNLOAD_DAEMON_LOG_EXCEPTIONS.mark();
+ numDownloadDaemonLogExceptions.mark();
throw e;
}
}
@@ -279,7 +303,7 @@ public class LogviewerResource {
return new JsonResponseBuilder().setData(UIHelpers.exceptionToJson(e, statusCode)).setCallback(callback)
.setStatus(statusCode).build();
} catch (IOException e) {
- ExceptionMeters.NUM_SEARCH_EXCEPTIONS.mark();
+ numSearchExceptions.mark();
throw e;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java
index 3c43249..0dbc95c 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java
@@ -18,6 +18,9 @@
package org.apache.storm.daemon.ui;
+import static org.apache.storm.utils.ConfigUtils.FILE_SEPARATOR;
+import static org.apache.storm.utils.ConfigUtils.STORM_HOME;
+
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.Arrays;
@@ -38,20 +41,18 @@ import org.apache.storm.security.auth.IHttpCredentialsPlugin;
import org.apache.storm.security.auth.ServerAuthUtils;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.Utils;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.servlet.DefaultServlet;
import org.eclipse.jetty.servlet.FilterHolder;
import org.eclipse.jetty.servlet.ServletContextHandler;
import org.eclipse.jetty.servlet.ServletHolder;
-
+import org.glassfish.hk2.utilities.binding.AbstractBinder;
import org.glassfish.jersey.server.ResourceConfig;
import org.glassfish.jersey.servlet.ServletContainer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.utils.ConfigUtils.FILE_SEPARATOR;
-import static org.apache.storm.utils.ConfigUtils.STORM_HOME;
-
/**
* Main class.
*
@@ -103,6 +104,8 @@ public class UIServer {
httpsTsPath, httpsTsPassword, httpsTsType, httpsNeedClientAuth, httpsWantClientAuth);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+
ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
context.setContextPath("/");
jettyServer.setHandler(context);
@@ -117,13 +120,19 @@ public class UIServer {
UIHelpers.configFilters(context, filterConfigurationList);
ResourceConfig resourceConfig =
- new ResourceConfig()
- .packages("org.apache.storm.daemon.ui.resources")
- .register(AuthorizedUserFilter.class)
- .register(HeaderResponseFilter.class)
- .register(AuthorizationExceptionMapper.class)
- .register(NotAliveExceptionMapper.class)
- .register(DefaultExceptionMapper.class);
+ new ResourceConfig()
+ .packages("org.apache.storm.daemon.ui.resources")
+ .registerInstances(new AbstractBinder() {
+ @Override
+ protected void configure() {
+ super.bind(metricsRegistry).to(StormMetricsRegistry.class);
+ }
+ })
+ .register(AuthorizedUserFilter.class)
+ .register(HeaderResponseFilter.class)
+ .register(AuthorizationExceptionMapper.class)
+ .register(NotAliveExceptionMapper.class)
+ .register(DefaultExceptionMapper.class);
ServletHolder jerseyServlet = new ServletHolder(new ServletContainer(resourceConfig));
jerseyServlet.setInitOrder(0);
@@ -155,7 +164,7 @@ public class UIServer {
holderHome.setInitParameter("dirAllowed","true");
holderHome.setInitParameter("pathInfoOnly","true");
- context.addFilter(new FilterHolder(new HeaderResponseServletFilter()), "/*", EnumSet.allOf(DispatcherType.class));
+ context.addFilter(new FilterHolder(new HeaderResponseServletFilter(metricsRegistry)), "/*", EnumSet.allOf(DispatcherType.class));
context.addServlet(holderHome,"/*");
@@ -164,7 +173,8 @@ public class UIServer {
holderPwd.setInitParameter("dirAllowed","true");
context.addServlet(holderPwd,"/");
- StormMetricsRegistry.startMetricsReporters(conf);
+ metricsRegistry.startMetricsReporters(conf);
+ Utils.addShutdownHookWithForceKillIn1Sec(metricsRegistry::stopMetricsReporters);
try {
jettyServer.start();
jettyServer.join();
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseFilter.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseFilter.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseFilter.java
index 962a45f..b935f64 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseFilter.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseFilter.java
@@ -20,6 +20,7 @@ package org.apache.storm.daemon.ui.filters;
import com.codahale.metrics.Meter;
import java.io.IOException;
+import javax.inject.Inject;
import javax.ws.rs.container.ContainerRequestContext;
import javax.ws.rs.container.ContainerResponseContext;
import javax.ws.rs.container.ContainerResponseFilter;
@@ -33,8 +34,12 @@ import org.slf4j.LoggerFactory;
public class HeaderResponseFilter implements ContainerResponseFilter {
public static final Logger LOG = LoggerFactory.getLogger(HeaderResponseFilter.class);
- public static Meter webRequestMeter =
- StormMetricsRegistry.registerMeter("num-web-requests");
+ private final Meter webRequestMeter;
+
+ @Inject
+ public HeaderResponseFilter(StormMetricsRegistry metricsRegistry) {
+ this.webRequestMeter = metricsRegistry.registerMeter("num-web-requests");
+ }
@Override
public void filter(ContainerRequestContext containerRequestContext,
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseServletFilter.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseServletFilter.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseServletFilter.java
index 84c424a..452431e 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseServletFilter.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/HeaderResponseServletFilter.java
@@ -35,11 +35,14 @@ import org.slf4j.LoggerFactory;
public class HeaderResponseServletFilter implements Filter {
public static final Logger LOG = LoggerFactory.getLogger(HeaderResponseServletFilter.class);
- public static Meter webRequestMeter =
- StormMetricsRegistry.registerMeter("num-web-requests");
+ private final Meter webRequestMeter;
- public static Meter mainPageRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-main-page-http-requests");
+ private final Meter mainPageRequestMeter;
+
+ public HeaderResponseServletFilter(StormMetricsRegistry metricsRegistry) {
+ this.webRequestMeter = metricsRegistry.registerMeter("num-web-requests");
+ this.mainPageRequestMeter = metricsRegistry.registerMeter("ui:num-main-page-http-requests");
+ }
@Override
public void init(FilterConfig filterConfig) throws ServletException {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
index f6c8c72..24c00ef 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
@@ -21,6 +21,7 @@ package org.apache.storm.daemon.ui.resources;
import com.codahale.metrics.Meter;
import java.net.URLDecoder;
import java.util.Map;
+import javax.inject.Inject;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
@@ -59,63 +60,48 @@ public class StormApiResource {
public static Map<String, Object> config = ConfigUtils.readStormConfig();
- public static Meter clusterConfigurationRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-cluster-configuration-http-requests");
-
- public static Meter clusterSummaryRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-cluster-summary-http-requests");
-
- public static Meter nimbusSummaryRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-nimbus-summary-http-requests");
-
- public static Meter supervisorRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-supervisor-http-requests");
-
- public static Meter supervisorSummaryRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-supervisor-summary-http-requests");
-
- public static Meter allTopologiesSummaryRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-all-topologies-summary-http-requests");
-
- public static Meter topologyPageRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-topology-page-http-requests");
-
- public static Meter topologyMetricRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-topology-metric-http-requests");
-
- public static Meter buildVisualizationRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-build-visualization-http-requests");
-
- public static Meter mkVisualizationDataRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-mk-visualization-data-http-requests");
-
- public static Meter componentPageRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-component-page-http-requests");
-
- public static Meter logConfigRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-log-config-http-requests");
-
- public static Meter activateTopologyRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-activate-topology-http-requests");
-
- public static Meter deactivateTopologyRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-deactivate-topology-http-requests");
-
- public static Meter debugTopologyRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-debug-topology-http-requests");
-
- public static Meter componentOpResponseRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-component-op-response-http-requests");
-
- public static Meter topologyOpResponseMeter =
- StormMetricsRegistry.registerMeter("ui:num-topology-op-response-http-requests");
-
- public static Meter topologyLagRequestMeter =
- StormMetricsRegistry.registerMeter("ui:num-topology-lag-http-requests");
-
- public static Meter getOwnerResourceSummariesMeter =
- StormMetricsRegistry.registerMeter("ui:num-get-owner-resource-summaries-http-request");
-
+ private final Meter clusterConfigurationRequestMeter;
+ private final Meter clusterSummaryRequestMeter;
+ private final Meter nimbusSummaryRequestMeter;
+ private final Meter supervisorRequestMeter;
+ private final Meter supervisorSummaryRequestMeter;
+ private final Meter allTopologiesSummaryRequestMeter;
+ private final Meter topologyPageRequestMeter;
+ private final Meter topologyMetricRequestMeter;
+ private final Meter buildVisualizationRequestMeter;
+ private final Meter mkVisualizationDataRequestMeter;
+ private final Meter componentPageRequestMeter;
+ private final Meter logConfigRequestMeter;
+ private final Meter activateTopologyRequestMeter;
+ private final Meter deactivateTopologyRequestMeter;
+ private final Meter debugTopologyRequestMeter;
+ private final Meter componentOpResponseRequestMeter;
+ private final Meter topologyOpResponseMeter;
+ private final Meter topologyLagRequestMeter;
+ private final Meter getOwnerResourceSummariesMeter;
+
+ @Inject
+ public StormApiResource(StormMetricsRegistry metricsRegistry) {
+ this.clusterConfigurationRequestMeter = metricsRegistry.registerMeter("ui:num-cluster-configuration-http-requests");
+ this.clusterSummaryRequestMeter = metricsRegistry.registerMeter("ui:num-cluster-summary-http-requests");
+ this.nimbusSummaryRequestMeter = metricsRegistry.registerMeter("ui:num-nimbus-summary-http-requests");
+ this.supervisorRequestMeter = metricsRegistry.registerMeter("ui:num-supervisor-http-requests");
+ this.supervisorSummaryRequestMeter = metricsRegistry.registerMeter("ui:num-supervisor-summary-http-requests");
+ this.allTopologiesSummaryRequestMeter = metricsRegistry.registerMeter("ui:num-all-topologies-summary-http-requests");
+ this.topologyPageRequestMeter = metricsRegistry.registerMeter("ui:num-topology-page-http-requests");
+ this.topologyMetricRequestMeter = metricsRegistry.registerMeter("ui:num-topology-metric-http-requests");
+ this.buildVisualizationRequestMeter = metricsRegistry.registerMeter("ui:num-build-visualization-http-requests");
+ this.mkVisualizationDataRequestMeter = metricsRegistry.registerMeter("ui:num-mk-visualization-data-http-requests");
+ this.componentPageRequestMeter = metricsRegistry.registerMeter("ui:num-component-page-http-requests");
+ this.logConfigRequestMeter = metricsRegistry.registerMeter("ui:num-log-config-http-requests");
+ this.activateTopologyRequestMeter = metricsRegistry.registerMeter("ui:num-activate-topology-http-requests");
+ this.deactivateTopologyRequestMeter = metricsRegistry.registerMeter("ui:num-deactivate-topology-http-requests");
+ this.debugTopologyRequestMeter = metricsRegistry.registerMeter("ui:num-debug-topology-http-requests");
+ this.componentOpResponseRequestMeter = metricsRegistry.registerMeter("ui:num-component-op-response-http-requests");
+ this.topologyOpResponseMeter = metricsRegistry.registerMeter("ui:num-topology-op-response-http-requests");
+ this.topologyLagRequestMeter = metricsRegistry.registerMeter("ui:num-topology-lag-http-requests");
+ this.getOwnerResourceSummariesMeter = metricsRegistry.registerMeter("ui:num-get-owner-resource-summaries-http-request");
+ }
/**
* /api/v1/cluster/configuration -> nimbus configuration.
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/test/java/org/apache/storm/daemon/drpc/DRPCServerTest.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/test/java/org/apache/storm/daemon/drpc/DRPCServerTest.java b/storm-webapp/src/test/java/org/apache/storm/daemon/drpc/DRPCServerTest.java
index a8452a9..aba8be8 100644
--- a/storm-webapp/src/test/java/org/apache/storm/daemon/drpc/DRPCServerTest.java
+++ b/storm-webapp/src/test/java/org/apache/storm/daemon/drpc/DRPCServerTest.java
@@ -37,6 +37,7 @@ import org.apache.storm.DaemonConfig;
import org.apache.storm.drpc.DRPCInvocationsClient;
import org.apache.storm.generated.DRPCExecutionException;
import org.apache.storm.generated.DRPCRequest;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.security.auth.SimpleTransportPlugin;
import org.apache.storm.utils.DRPCClient;
import org.junit.AfterClass;
@@ -88,7 +89,7 @@ public class DRPCServerTest {
@Test
public void testGoodThrift() throws Exception {
Map<String, Object> conf = getConf(0, 0, null);
- try (DRPCServer server = new DRPCServer(conf)) {
+ try (DRPCServer server = new DRPCServer(conf, new StormMetricsRegistry())) {
server.start();
try (DRPCClient client = new DRPCClient(conf, "localhost", server.getDrpcPort());
DRPCInvocationsClient invoke = new DRPCInvocationsClient(conf, "localhost", server.getDrpcInvokePort())) {
@@ -107,7 +108,7 @@ public class DRPCServerTest {
@Test
public void testFailedThrift() throws Exception {
Map<String, Object> conf = getConf(0, 0, null);
- try (DRPCServer server = new DRPCServer(conf)) {
+ try (DRPCServer server = new DRPCServer(conf, new StormMetricsRegistry())) {
server.start();
try (DRPCClient client = new DRPCClient(conf, "localhost", server.getDrpcPort());
DRPCInvocationsClient invoke = new DRPCInvocationsClient(conf, "localhost", server.getDrpcInvokePort())) {
@@ -146,7 +147,7 @@ public class DRPCServerTest {
public void testGoodHttpGet() throws Exception {
LOG.info("STARTING HTTP GET TEST...");
Map<String, Object> conf = getConf(0, 0, 0);
- try (DRPCServer server = new DRPCServer(conf)) {
+ try (DRPCServer server = new DRPCServer(conf, new StormMetricsRegistry())) {
server.start();
//TODO need a better way to do this
Thread.sleep(2000);
@@ -167,7 +168,7 @@ public class DRPCServerTest {
public void testFailedHttpGet() throws Exception {
LOG.info("STARTING HTTP GET (FAIL) TEST...");
Map<String, Object> conf = getConf(0, 0, 0);
- try (DRPCServer server = new DRPCServer(conf)) {
+ try (DRPCServer server = new DRPCServer(conf, new StormMetricsRegistry())) {
server.start();
//TODO need a better way to do this
Thread.sleep(2000);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandlerTest.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandlerTest.java b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandlerTest.java
index 58daa30..76ff381 100644
--- a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandlerTest.java
+++ b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandlerTest.java
@@ -35,6 +35,7 @@ import javax.ws.rs.core.Response;
import org.apache.storm.daemon.logviewer.utils.LogviewerResponseBuilder;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
import org.assertj.core.util.Lists;
import org.junit.Test;
@@ -61,8 +62,9 @@ public class LogviewerLogPageHandlerTest {
String origin = "www.origin.server.net";
Map<String, Object> stormConf = Utils.readStormConfig();
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
LogviewerLogPageHandler handler = new LogviewerLogPageHandler(rootPath, null,
- new WorkerLogs(stormConf, new File(rootPath)), new ResourceAuthorizer(stormConf));
+ new WorkerLogs(stormConf, new File(rootPath), metricsRegistry), new ResourceAuthorizer(stormConf), metricsRegistry);
final Response expectedAll = LogviewerResponseBuilder.buildSuccessJsonResponse(
Lists.newArrayList("topoA/port1/worker.log", "topoA/port2/worker.log", "topoB/port1/worker.log"),
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandlerTest.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandlerTest.java b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandlerTest.java
index 0a450be..43c4134 100644
--- a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandlerTest.java
+++ b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandlerTest.java
@@ -53,6 +53,7 @@ import org.apache.storm.DaemonConfig;
import org.apache.storm.daemon.logviewer.LogviewerConstant;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.ui.InvalidRequestException;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
import org.jooq.lambda.Seq;
import org.jooq.lambda.Unchecked;
@@ -849,7 +850,7 @@ public class LogviewerLogSearchHandlerTest {
private LogviewerLogSearchHandler getStubbedSearchHandler() {
Map<String, Object> stormConf = Utils.readStormConfig();
LogviewerLogSearchHandler handler = new LogviewerLogSearchHandler(stormConf, topoPath, null,
- new ResourceAuthorizer(stormConf));
+ new ResourceAuthorizer(stormConf), new StormMetricsRegistry());
handler = spy(handler);
doReturn(logFiles).when(handler).logsForPort(any(), any());
@@ -868,14 +869,14 @@ public class LogviewerLogSearchHandlerTest {
private static LogviewerLogSearchHandler getSearchHandler() {
Map<String, Object> stormConf = Utils.readStormConfig();
return new LogviewerLogSearchHandler(stormConf, null, null,
- new ResourceAuthorizer(stormConf));
+ new ResourceAuthorizer(stormConf), new StormMetricsRegistry());
}
private static LogviewerLogSearchHandler getSearchHandlerWithPort(int port) {
Map<String, Object> stormConf = Utils.readStormConfig();
stormConf.put(DaemonConfig.LOGVIEWER_PORT, port);
return new LogviewerLogSearchHandler(stormConf, null, null,
- new ResourceAuthorizer(stormConf));
+ new ResourceAuthorizer(stormConf), new StormMetricsRegistry());
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/LogCleanerTest.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/LogCleanerTest.java b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/LogCleanerTest.java
index 8b1c0b4..d85edea 100644
--- a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/LogCleanerTest.java
+++ b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/LogCleanerTest.java
@@ -55,6 +55,7 @@ import org.apache.storm.daemon.logviewer.testsupport.MockDirectoryBuilder;
import org.apache.storm.daemon.logviewer.testsupport.MockRemovableFileBuilder;
import org.apache.storm.daemon.supervisor.SupervisorUtils;
import org.apache.storm.generated.LSWorkerHeartbeat;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Time;
import org.apache.storm.utils.Utils;
import org.jooq.lambda.Seq;
@@ -79,9 +80,10 @@ public class LogCleanerTest {
conf.put(LOGVIEWER_CLEANUP_AGE_MINS, 60);
conf.put(LOGVIEWER_CLEANUP_INTERVAL_SECS, 300);
- WorkerLogs workerLogs = new WorkerLogs(conf, null);
+ StormMetricsRegistry metricRegistry = new StormMetricsRegistry();
+ WorkerLogs workerLogs = new WorkerLogs(conf, null, metricRegistry);
- LogCleaner logCleaner = new LogCleaner(conf, workerLogs, mockDirectoryCleaner, null);
+ LogCleaner logCleaner = new LogCleaner(conf, workerLogs, mockDirectoryCleaner, null, metricRegistry);
final long nowMillis = Time.currentTimeMillis();
final long cutoffMillis = logCleaner.cleanupCutoffAgeMillis(nowMillis);
@@ -156,8 +158,9 @@ public class LogCleanerTest {
.setFiles(rootFiles).build();
Map<String, Object> conf = Utils.readStormConfig();
- WorkerLogs workerLogs = new WorkerLogs(conf, rootDir);
- LogCleaner logCleaner = new LogCleaner(conf, workerLogs, mockDirectoryCleaner, rootDir);
+ StormMetricsRegistry metricRegistry = new StormMetricsRegistry();
+ WorkerLogs workerLogs = new WorkerLogs(conf, rootDir, metricRegistry);
+ LogCleaner logCleaner = new LogCleaner(conf, workerLogs, mockDirectoryCleaner, rootDir, metricRegistry);
List<Integer> deletedFiles = logCleaner.perWorkerDirCleanup(1200)
.stream()
@@ -219,14 +222,15 @@ public class LogCleanerTest {
.setFiles(rootFiles).build();
Map<String, Object> conf = Utils.readStormConfig();
- WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, rootDir) {
+ StormMetricsRegistry metricRegistry = new StormMetricsRegistry();
+ WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, rootDir, metricRegistry) {
@Override
public SortedSet<String> getAliveWorkerDirs() {
return new TreeSet<>(Collections.singletonList("/workers-artifacts/topo1/port1"));
}
};
- LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, mockDirectoryCleaner, rootDir);
+ LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, mockDirectoryCleaner, rootDir, metricRegistry);
int deletedFiles = logCleaner.globalLogCleanup(2400).deletedFiles;
assertEquals(18, deletedFiles);
} finally {
@@ -257,7 +261,8 @@ public class LogCleanerTest {
SupervisorUtils.setInstance(mockedSupervisorUtils);
Map<String, Object> conf = Utils.readStormConfig();
- WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, null) {
+ StormMetricsRegistry metricRegistry = new StormMetricsRegistry();
+ WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, null, metricRegistry) {
@Override
public Map<String, File> identifyWorkerLogDirs(Set<File> logDirs) {
Map<String, File> ret = new HashMap<>();
@@ -270,7 +275,7 @@ public class LogCleanerTest {
}
};
- LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, new DirectoryCleaner(), null);
+ LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, new DirectoryCleaner(metricRegistry), null, metricRegistry);
when(mockedSupervisorUtils.readWorkerHeartbeatsImpl(anyMapOf(String.class, Object.class))).thenReturn(idToHb);
assertEquals(Sets.newSet(expectedDir2, expectedDir3), logCleaner.getDeadWorkerDirs(nowSecs, logDirs));
@@ -301,9 +306,10 @@ public class LogCleanerTest {
Map<String, Object> conf = Utils.readStormConfig();
- WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, null);
+ StormMetricsRegistry metricRegistry = new StormMetricsRegistry();
+ WorkerLogs stubbedWorkerLogs = new WorkerLogs(conf, null, metricRegistry);
- LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, new DirectoryCleaner(), null) {
+ LogCleaner logCleaner = new LogCleaner(conf, stubbedWorkerLogs, new DirectoryCleaner(metricRegistry), null, metricRegistry) {
@Override
Set<File> selectDirsForCleanup(long nowMillis) {
return Collections.emptySet();
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/WorkerLogsTest.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/WorkerLogsTest.java b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/WorkerLogsTest.java
index 30b1678..c7ca3dd 100644
--- a/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/WorkerLogsTest.java
+++ b/storm-webapp/src/test/java/org/apache/storm/daemon/logviewer/utils/WorkerLogsTest.java
@@ -32,6 +32,7 @@ import java.util.Optional;
import org.apache.storm.daemon.logviewer.testsupport.MockDirectoryBuilder;
import org.apache.storm.daemon.logviewer.testsupport.MockFileBuilder;
import org.apache.storm.daemon.supervisor.SupervisorUtils;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
import org.junit.Test;
@@ -53,7 +54,7 @@ public class WorkerLogsTest {
SupervisorUtils.setInstance(mockedSupervisorUtils);
Map<String, Object> stormConf = Utils.readStormConfig();
- WorkerLogs workerLogs = new WorkerLogs(stormConf, port1Dir) {
+ WorkerLogs workerLogs = new WorkerLogs(stormConf, port1Dir, new StormMetricsRegistry()) {
@Override
public Optional<File> getMetadataFileForWorkerLogDir(File logDir) throws IOException {
return Optional.of(mockMetaFile);
[6/6] storm git commit: Merge branch 'STORM-3197' of
https://github.com/srdo/storm into STORM-3197
Posted by bo...@apache.org.
Merge branch 'STORM-3197' of https://github.com/srdo/storm into STORM-3197
STORM-3197: Make StormMetricsRegistry non-static
This closes #2805
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/eaed3cbf
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/eaed3cbf
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/eaed3cbf
Branch: refs/heads/master
Commit: eaed3cbf3610bbefb9466f2c40db1e0e2a20ca9c
Parents: 37d22b8 8c90f12
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Mon Sep 17 14:31:28 2018 -0500
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Mon Sep 17 14:31:28 2018 -0500
----------------------------------------------------------------------
.../storm/cassandra/trident/MapStateTest.java | 7 +-
pom.xml | 2 +-
.../jvm/org/apache/storm/utils/ShellUtils.java | 1 -
.../org/apache/storm/command/KillWorkers.java | 5 +-
.../apache/storm/trident/integration_test.clj | 18 +-
.../test/clj/org/apache/storm/drpc_test.clj | 20 +-
.../test/clj/org/apache/storm/nimbus_test.clj | 21 +-
.../scheduler/multitenant_scheduler_test.clj | 38 ++--
.../clj/org/apache/storm/scheduler_test.clj | 3 +
.../java/org/apache/storm/LocalCluster.java | 21 +-
.../main/java/org/apache/storm/LocalDRPC.java | 14 +-
.../java/org/apache/storm/daemon/drpc/DRPC.java | 22 +-
.../reporters/ConsolePreparableReporter.java | 3 +-
.../reporters/CsvPreparableReporter.java | 3 +-
.../reporters/JmxPreparableReporter.java | 2 +-
.../metrics/reporters/PreparableReporter.java | 4 +-
.../org/apache/storm/daemon/nimbus/Nimbus.java | 227 ++++++++++++-------
.../storm/daemon/supervisor/BasicContainer.java | 22 +-
.../supervisor/BasicContainerLauncher.java | 15 +-
.../storm/daemon/supervisor/Container.java | 154 +++++--------
.../daemon/supervisor/ContainerLauncher.java | 14 +-
.../supervisor/ContainerMemoryTracker.java | 160 +++++++++++++
.../storm/daemon/supervisor/LocalContainer.java | 7 +-
.../supervisor/LocalContainerLauncher.java | 11 +-
.../daemon/supervisor/ReadClusterState.java | 7 +-
.../daemon/supervisor/RunAsUserContainer.java | 15 +-
.../supervisor/RunAsUserContainerLauncher.java | 16 +-
.../apache/storm/daemon/supervisor/Slot.java | 64 +++---
.../storm/daemon/supervisor/SlotMetrics.java | 46 ++++
.../storm/daemon/supervisor/Supervisor.java | 44 +++-
.../apache/storm/localizer/AsyncLocalizer.java | 36 +--
.../storm/localizer/LocalizedResource.java | 5 +-
.../storm/localizer/LocallyCachedBlob.java | 7 +-
.../localizer/LocallyCachedTopologyBlob.java | 9 +-
.../storm/metric/StormMetricsRegistry.java | 104 +++------
.../apache/storm/metricstore/MetricStore.java | 4 +-
.../storm/metricstore/MetricStoreConfig.java | 6 +-
.../metricstore/rocksdb/MetricsCleaner.java | 12 +-
.../storm/metricstore/rocksdb/RocksDbStore.java | 8 +-
.../storm/nimbus/LeaderListenerCallback.java | 10 +-
.../org/apache/storm/pacemaker/Pacemaker.java | 29 ++-
.../org/apache/storm/scheduler/Cluster.java | 17 +-
.../storm/scheduler/SupervisorDetails.java | 1 +
.../apache/storm/scheduler/TopologyDetails.java | 2 -
.../scheduler/blacklist/BlacklistScheduler.java | 6 +-
.../strategies/RasBlacklistStrategy.java | 4 +-
.../storm/scheduler/resource/RAS_Node.java | 2 +-
.../storm/scheduler/resource/ResourceUtils.java | 9 +-
.../normalization/NormalizedResourceOffer.java | 12 +-
.../NormalizedResourceRequest.java | 2 +-
.../normalization/NormalizedResources.java | 12 +-
.../resource/normalization/ResourceMetrics.java | 34 +++
.../scheduling/BaseResourceAwareStrategy.java | 13 +-
.../storm/zookeeper/LeaderElectorImp.java | 9 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 14 +-
.../java/org/apache/storm/PacemakerTest.java | 12 +-
.../org/apache/storm/daemon/drpc/DRPCTest.java | 10 +-
.../daemon/supervisor/BasicContainerTest.java | 43 ++--
.../storm/daemon/supervisor/ContainerTest.java | 12 +-
.../storm/daemon/supervisor/SlotTest.java | 39 ++--
.../storm/localizer/AsyncLocalizerTest.java | 11 +-
.../LocalizedResourceRetentionSetTest.java | 14 +-
.../storm/metric/StormMetricsRegistryTest.java | 111 ---------
.../metricstore/rocksdb/RocksDbStoreTest.java | 5 +-
.../scheduler/blacklist/FaultGenerateUtils.java | 4 +-
.../blacklist/TestBlacklistScheduler.java | 66 +++---
.../resource/TestResourceAwareScheduler.java | 51 +++--
.../storm/scheduler/resource/TestUser.java | 5 +-
.../NormalizedResourceOfferTest.java | 5 +-
.../normalization/NormalizedResourcesTest.java | 9 +-
.../eviction/TestDefaultEvictionStrategy.java | 13 +-
.../TestFIFOSchedulingPriorityStrategy.java | 9 +-
.../TestConstraintSolverStrategy.java | 9 +-
.../TestDefaultResourceAwareStrategy.java | 10 +-
.../TestGenericResourceAwareStrategy.java | 9 +-
.../apache/storm/daemon/drpc/DRPCServer.java | 24 +-
.../daemon/drpc/webapp/DRPCApplication.java | 7 +-
.../storm/daemon/drpc/webapp/DRPCResource.java | 12 +-
.../storm/daemon/logviewer/LogviewerServer.java | 35 +--
.../handler/LogviewerLogDownloadHandler.java | 7 +-
.../handler/LogviewerLogPageHandler.java | 31 ++-
.../handler/LogviewerLogSearchHandler.java | 29 ++-
.../handler/LogviewerProfileHandler.java | 16 +-
.../logviewer/utils/DirectoryCleaner.java | 16 +-
.../logviewer/utils/ExceptionMeterNames.java | 60 +++++
.../daemon/logviewer/utils/ExceptionMeters.java | 66 ------
.../daemon/logviewer/utils/LogCleaner.java | 26 ++-
.../logviewer/utils/LogFileDownloader.java | 15 +-
.../utils/LogviewerResponseBuilder.java | 5 +-
.../daemon/logviewer/utils/WorkerLogs.java | 15 +-
.../logviewer/webapp/LogviewerApplication.java | 22 +-
.../logviewer/webapp/LogviewerResource.java | 76 ++++---
.../org/apache/storm/daemon/ui/UIServer.java | 36 +--
.../daemon/ui/filters/HeaderResponseFilter.java | 9 +-
.../ui/filters/HeaderResponseServletFilter.java | 11 +-
.../daemon/ui/resources/StormApiResource.java | 100 ++++----
.../storm/daemon/drpc/DRPCServerTest.java | 9 +-
.../handler/LogviewerLogPageHandlerTest.java | 4 +-
.../handler/LogviewerLogSearchHandlerTest.java | 7 +-
.../daemon/logviewer/utils/LogCleanerTest.java | 26 ++-
.../daemon/logviewer/utils/WorkerLogsTest.java | 3 +-
101 files changed, 1434 insertions(+), 1033 deletions(-)
----------------------------------------------------------------------
[4/6] storm git commit: STORM-3197: Make StormMetricsRegistry
non-static
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
index 907c717..8ad5936 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Container.java
@@ -33,8 +33,8 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
import org.apache.storm.Config;
import org.apache.storm.DaemonConfig;
import org.apache.storm.container.ResourceIsolationInterface;
@@ -60,47 +60,19 @@ import org.yaml.snakeyaml.Yaml;
* Represents a container that a worker will run in.
*/
public abstract class Container implements Killable {
+
private static final Logger LOG = LoggerFactory.getLogger(Container.class);
private static final String MEMORY_USED_METRIC = "UsedMemory";
private static final String SYSTEM_COMPONENT_ID = "System";
private static final String INVALID_EXECUTOR_ID = "-1";
private static final String INVALID_STREAM_ID = "None";
- private static final ConcurrentHashMap<Integer, TopoAndMemory> _usedMemory =
- new ConcurrentHashMap<>();
- private static final ConcurrentHashMap<Integer, TopoAndMemory> _reservedMemory =
- new ConcurrentHashMap<>();
-
- private static final Meter numCleanupExceptions = StormMetricsRegistry.registerMeter("supervisor:num-cleanup-exceptions");
- private static final Meter numKillExceptions = StormMetricsRegistry.registerMeter("supervisor:num-kill-exceptions");
- private static final Meter numForceKillExceptions = StormMetricsRegistry.registerMeter("supervisor:num-force-kill-exceptions");
- private static final Meter numForceKill = StormMetricsRegistry.registerMeter("supervisor:num-workers-force-kill");
- private static final Timer shutdownDuration = StormMetricsRegistry.registerTimer("supervisor:worker-shutdown-duration-ns");
- private static final Timer cleanupDuration = StormMetricsRegistry.registerTimer("supervisor:worker-per-call-clean-up-duration-ns");
-
- static {
- StormMetricsRegistry.registerGauge(
- "supervisor:current-used-memory-mb",
- () -> {
- Long val =
- _usedMemory.values().stream().mapToLong((topoAndMem) -> topoAndMem.memory).sum();
- int ret = val.intValue();
- if (val > Integer.MAX_VALUE) { // Would only happen at 2 PB so we are OK for now
- ret = Integer.MAX_VALUE;
- }
- return ret;
- });
- StormMetricsRegistry.registerGauge(
- "supervisor:current-reserved-memory-mb",
- () -> {
- Long val =
- _reservedMemory.values().stream().mapToLong((topoAndMem) -> topoAndMem.memory).sum();
- int ret = val.intValue();
- if (val > Integer.MAX_VALUE) { // Would only happen at 2 PB so we are OK for now
- ret = Integer.MAX_VALUE;
- }
- return ret;
- });
- }
+
+ private final Meter numCleanupExceptions;
+ private final Meter numKillExceptions;
+ private final Meter numForceKillExceptions;
+ private final Meter numForceKill;
+ private final Timer shutdownDuration;
+ private final Timer cleanupDuration;
protected final Map<String, Object> _conf;
protected final Map<String, Object> _topoConf; //Not set if RECOVER_PARTIAL
@@ -114,28 +86,31 @@ public abstract class Container implements Killable {
protected final boolean _symlinksDisabled;
protected String _workerId;
protected ContainerType _type;
+ protected ContainerMemoryTracker containerMemoryTracker;
private long lastMetricProcessTime = 0L;
private Timer.Context shutdownTimer = null;
/**
* Create a new Container.
*
- * @param type the type of container being made.
- * @param conf the supervisor config
- * @param supervisorId the ID of the supervisor this is a part of.
- * @param supervisorPort the thrift server port of the supervisor this is a part of.
- * @param port the port the container is on. Should be <= 0 if only a partial recovery
- * @param assignment the assignment for this container. Should be null if only a partial recovery.
+ * @param type the type of container being made.
+ * @param conf the supervisor config
+ * @param supervisorId the ID of the supervisor this is a part of.
+ * @param supervisorPort the thrift server port of the supervisor this is a part of.
+ * @param port the port the container is on. Should be <= 0 if only a partial recovery @param assignment
+ * the assignment for this container. Should be null if only a partial recovery.
* @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
- * @param workerId the id of the worker to use. Must not be null if doing a partial recovery.
- * @param topoConf the config of the topology (mostly for testing) if null and not a partial recovery the real conf is
- * read.
- * @param ops file system operations (mostly for testing) if null a new one is made
+ * @param workerId the id of the worker to use. Must not be null if doing a partial recovery.
+ * @param topoConf the config of the topology (mostly for testing) if null and not a partial recovery the real conf is read.
+ * @param ops file system operations (mostly for testing) if null a new one is made
+ * @param metricsRegistry The metrics registry.
+ * @param containerMemoryTracker The shared memory tracker for the supervisor's containers
* @throws IOException on any error.
*/
protected Container(ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort,
- int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- String workerId, Map<String, Object> topoConf, AdvancedFSOps ops) throws IOException {
+ int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
+ String workerId, Map<String, Object> topoConf, AdvancedFSOps ops,
+ StormMetricsRegistry metricsRegistry, ContainerMemoryTracker containerMemoryTracker) throws IOException {
assert (type != null);
assert (conf != null);
assert (supervisorId != null);
@@ -180,6 +155,13 @@ public abstract class Container implements Killable {
_topoConf = topoConf;
}
}
+ this.numCleanupExceptions = metricsRegistry.registerMeter("supervisor:num-cleanup-exceptions");
+ this.numKillExceptions = metricsRegistry.registerMeter("supervisor:num-kill-exceptions");
+ this.numForceKillExceptions = metricsRegistry.registerMeter("supervisor:num-force-kill-exceptions");
+ this.numForceKill = metricsRegistry.registerMeter("supervisor:num-workers-force-kill");
+ this.shutdownDuration = metricsRegistry.registerTimer("supervisor:worker-shutdown-duration-ns");
+ this.cleanupDuration = metricsRegistry.registerTimer("supervisor:worker-per-call-clean-up-duration-ns");
+ this.containerMemoryTracker = containerMemoryTracker;
}
@Override
@@ -263,7 +245,7 @@ public abstract class Container implements Killable {
/**
* Is a process alive and running?.
*
- * @param pid the PID of the running process
+ * @param pid the PID of the running process
* @param user the user that is expected to own that process
* @return true if it is, else false
*
@@ -298,7 +280,7 @@ public abstract class Container implements Killable {
}
} else {
LOG.error("Received unexpected output from tasklist command. Expected one colon in user name line. Line was {}",
- read);
+ read);
}
break;
}
@@ -332,9 +314,9 @@ public abstract class Container implements Killable {
public boolean areAllProcessesDead() throws IOException {
Set<Long> pids = getAllPids();
String user = getRunWorkerAsUser();
-
+
boolean allDead = true;
- for (Long pid: pids) {
+ for (Long pid : pids) {
LOG.debug("Checking if pid {} owner {} is alive", pid, user);
if (!isProcessAlive(pid, user)) {
LOG.debug("{}: PID {} is dead", _workerId, pid);
@@ -355,8 +337,7 @@ public abstract class Container implements Killable {
@Override
public void cleanUp() throws IOException {
try (Timer.Context t = cleanupDuration.time()) {
- _usedMemory.remove(_port);
- _reservedMemory.remove(_port);
+ containerMemoryTracker.remove(_port);
cleanUpForRestart();
} catch (IOException e) {
//This may or may not be reported depending on when process exits
@@ -366,7 +347,7 @@ public abstract class Container implements Killable {
}
/**
- * Setup the container to run. By default this creates the needed directories/links in the local file system PREREQUISITE: All needed
+ * Setup the container to run. By default this creates the needed directories/links in the local file system PREREQUISITE: All needed
* blobs and topology, jars/configs have been downloaded and placed in the appropriate locations
*
* @throws IOException on any error
@@ -375,8 +356,8 @@ public abstract class Container implements Killable {
_type.assertFull();
if (!_ops.doRequiredTopoFilesExist(_conf, _topologyId)) {
LOG.info("Missing topology storm code, so can't launch worker with assignment {} for this supervisor {} on port {} with id {}",
- _assignment,
- _supervisorId, _port, _workerId);
+ _assignment,
+ _supervisorId, _port, _workerId);
throw new IllegalStateException("Not all needed files are here!!!!");
}
LOG.info("Setting up {}:{}", _supervisorId, _workerId);
@@ -499,16 +480,16 @@ public abstract class Container implements Killable {
if (!_symlinksDisabled) {
LOG.info("Creating symlinks for worker-id: {} storm-id: {} for files({}): {}", _workerId, _topologyId, resourceFileNames.size(),
- resourceFileNames);
+ resourceFileNames);
if (targetResourcesDir.exists()) {
_ops.createSymlink(new File(workerRoot, ServerConfigUtils.RESOURCES_SUBDIR), targetResourcesDir);
} else {
LOG.info("Topology jar for worker-id: {} storm-id: {} does not contain re sources directory {}.", _workerId, _topologyId,
- targetResourcesDir.toString());
+ targetResourcesDir.toString());
}
for (String fileName : blobFileNames) {
_ops.createSymlink(new File(workerRoot, fileName),
- new File(stormRoot, fileName));
+ new File(stormRoot, fileName));
}
} else if (blobFileNames.size() > 0) {
LOG.warn("Symlinks are disabled, no symlinks created for blobs {}", blobFileNames);
@@ -626,8 +607,8 @@ public abstract class Container implements Killable {
_type.assertFull();
long used = getMemoryUsageMb();
long reserved = getMemoryReservationMb();
- _usedMemory.put(_port, new TopoAndMemory(_topologyId, used));
- _reservedMemory.put(_port, new TopoAndMemory(_topologyId, reserved));
+ containerMemoryTracker.setUsedMemoryMb(_port, _topologyId, used);
+ containerMemoryTracker.setReservedMemoryMb(_port, _topologyId, reserved);
}
/**
@@ -635,12 +616,7 @@ public abstract class Container implements Killable {
*/
public long getTotalTopologyMemoryUsed() {
updateMemoryAccounting();
- return _usedMemory
- .values()
- .stream()
- .filter((topoAndMem) -> _topologyId.equals(topoAndMem.topoId))
- .mapToLong((topoAndMem) -> topoAndMem.memory)
- .sum();
+ return containerMemoryTracker.getUsedMemoryMb(_topologyId);
}
/**
@@ -652,12 +628,7 @@ public abstract class Container implements Killable {
public long getTotalTopologyMemoryReserved(LocalAssignment withUpdatedLimits) {
updateMemoryAccounting();
long ret =
- _reservedMemory
- .values()
- .stream()
- .filter((topoAndMem) -> _topologyId.equals(topoAndMem.topoId))
- .mapToLong((topoAndMem) -> topoAndMem.memory)
- .sum();
+ containerMemoryTracker.getReservedMemoryMb(_topologyId);
if (withUpdatedLimits.is_set_total_node_shared()) {
ret += withUpdatedLimits.get_total_node_shared();
}
@@ -668,11 +639,7 @@ public abstract class Container implements Killable {
* Get the number of workers for this topology.
*/
public long getTotalWorkersForThisTopology() {
- return _usedMemory
- .values()
- .stream()
- .filter((topoAndMem) -> _topologyId.equals(topoAndMem.topoId))
- .count();
+ return containerMemoryTracker.getAssignedWorkerCount(_topologyId);
}
/**
@@ -712,10 +679,10 @@ public abstract class Container implements Killable {
* Run a profiling request.
*
* @param request the request to run
- * @param stop is this a stop request?
+ * @param stop is this a stop request?
* @return true if it succeeded, else false
*
- * @throws IOException on any error
+ * @throws IOException on any error
* @throws InterruptedException if running the command is interrupted.
*/
public abstract boolean runProfiling(ProfileRequest request, boolean stop) throws IOException, InterruptedException;
@@ -732,7 +699,8 @@ public abstract class Container implements Killable {
*/
void processMetrics(OnlyLatestExecutor<Integer> exec, WorkerMetricsProcessor processor) {
try {
- if (_usedMemory.get(_port) != null) {
+ Optional<Long> usedMemoryForPort = containerMemoryTracker.getUsedMemoryMb(_port);
+ if (usedMemoryForPort.isPresent()) {
// Make sure we don't process too frequently.
long nextMetricProcessTime = this.lastMetricProcessTime + 60L * 1000L;
long currentTimeMsec = System.currentTimeMillis();
@@ -744,9 +712,8 @@ public abstract class Container implements Killable {
// create metric for memory
long timestamp = System.currentTimeMillis();
- double value = _usedMemory.get(_port).memory;
- WorkerMetricPoint workerMetric = new WorkerMetricPoint(MEMORY_USED_METRIC, timestamp, value, SYSTEM_COMPONENT_ID,
- INVALID_EXECUTOR_ID, INVALID_STREAM_ID);
+ WorkerMetricPoint workerMetric = new WorkerMetricPoint(MEMORY_USED_METRIC, timestamp, usedMemoryForPort.get(), SYSTEM_COMPONENT_ID,
+ INVALID_EXECUTOR_ID, INVALID_STREAM_ID);
WorkerMetricList metricList = new WorkerMetricList();
metricList.add_to_metrics(workerMetric);
@@ -794,19 +761,4 @@ public abstract class Container implements Killable {
return _onlyKillable;
}
}
-
- private static class TopoAndMemory {
- public final String topoId;
- public final long memory;
-
- public TopoAndMemory(String id, long mem) {
- topoId = id;
- memory = mem;
- }
-
- @Override
- public String toString() {
- return "{TOPO: " + topoId + " at " + memory + " MB}";
- }
- }
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java
index 2f32e38..7df2036 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerLauncher.java
@@ -19,6 +19,7 @@ import org.apache.storm.DaemonConfig;
import org.apache.storm.container.ResourceIsolationInterface;
import org.apache.storm.generated.LocalAssignment;
import org.apache.storm.messaging.IContext;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.LocalState;
import org.apache.storm.utils.ObjectReader;
@@ -43,13 +44,16 @@ public abstract class ContainerLauncher {
* @param supervisorId the ID of the supervisor
* @param supervisorPort the parent supervisor thrift server port
* @param sharedContext Used in local mode to let workers talk together without netty
+ * @param metricsRegistry The metrics registry.
+ * @param containerMemoryTracker The shared memory tracker for the supervisor's containers
* @return the proper container launcher
* @throws IOException on any error
*/
public static ContainerLauncher make(Map<String, Object> conf, String supervisorId, int supervisorPort,
- IContext sharedContext) throws IOException {
+ IContext sharedContext, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
if (ConfigUtils.isLocalMode(conf)) {
- return new LocalContainerLauncher(conf, supervisorId, supervisorPort, sharedContext);
+ return new LocalContainerLauncher(conf, supervisorId, supervisorPort, sharedContext, metricsRegistry, containerMemoryTracker);
}
ResourceIsolationInterface resourceIsolationManager = null;
@@ -61,9 +65,11 @@ public abstract class ContainerLauncher {
}
if (ObjectReader.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
- return new RunAsUserContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager);
+ return new RunAsUserContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager, metricsRegistry,
+ containerMemoryTracker);
}
- return new BasicContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager);
+ return new BasicContainerLauncher(conf, supervisorId, supervisorPort, resourceIsolationManager, metricsRegistry,
+ containerMemoryTracker);
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerMemoryTracker.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerMemoryTracker.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerMemoryTracker.java
new file mode 100644
index 0000000..b5fc1f8
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ContainerMemoryTracker.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2018 The Apache Software Foundation.
+ *
+ * Licensed 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.storm.daemon.supervisor;
+
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.storm.metric.StormMetricsRegistry;
+
+public class ContainerMemoryTracker {
+
+ private final ConcurrentHashMap<Integer, TopoAndMemory> usedMemory =
+ new ConcurrentHashMap<>();
+ private final ConcurrentHashMap<Integer, TopoAndMemory> reservedMemory =
+ new ConcurrentHashMap<>();
+
+ public ContainerMemoryTracker(StormMetricsRegistry metricsRegistry) {
+ metricsRegistry.registerGauge(
+ "supervisor:current-used-memory-mb",
+ () -> {
+ Long val =
+ usedMemory.values().stream().mapToLong((topoAndMem) -> topoAndMem.memory).sum();
+ int ret = val.intValue();
+ if (val > Integer.MAX_VALUE) { // Would only happen at 2 PB so we are OK for now
+ ret = Integer.MAX_VALUE;
+ }
+ return ret;
+ });
+ metricsRegistry.registerGauge(
+ "supervisor:current-reserved-memory-mb",
+ () -> {
+ Long val =
+ reservedMemory.values().stream().mapToLong((topoAndMem) -> topoAndMem.memory).sum();
+ int ret = val.intValue();
+ if (val > Integer.MAX_VALUE) { // Would only happen at 2 PB so we are OK for now
+ ret = Integer.MAX_VALUE;
+ }
+ return ret;
+ });
+ }
+
+ /**
+ * Get the memory used by the worker on the given port.
+ *
+ * @param port The worker port
+ * @return The memory used by the worker, or empty if no worker exists on the given port.
+ */
+ public Optional<Long> getUsedMemoryMb(int port) {
+ TopoAndMemory topoAndMemory = usedMemory.get(port);
+ if (topoAndMemory == null) {
+ return Optional.empty();
+ }
+ return Optional.of(topoAndMemory.memory);
+ }
+
+ /**
+ * Gets the memory used by the given topology across all ports on this supervisor.
+ *
+ * @param topologyId The topology id
+ * @return The memory used by the given topology id
+ */
+ public long getUsedMemoryMb(String topologyId) {
+ return usedMemory
+ .values()
+ .stream()
+ .filter((topoAndMem) -> topologyId.equals(topoAndMem.topoId))
+ .mapToLong((topoAndMem) -> topoAndMem.memory)
+ .sum();
+ }
+
+ /**
+ * Gets the memory reserved by the given topology across all ports on this supervisor.
+ *
+ * @param topologyId The topology id
+ * @return The memory reserved by the given topology id
+ */
+ public long getReservedMemoryMb(String topologyId) {
+ return reservedMemory
+ .values()
+ .stream()
+ .filter((topoAndMem) -> topologyId.equals(topoAndMem.topoId))
+ .mapToLong((topoAndMem) -> topoAndMem.memory)
+ .sum();
+ }
+
+ /**
+ * Gets the number of worker ports assigned to the given topology id on this supervisor.
+ *
+ * @param topologyId The topology id
+ * @return The number of worker ports assigned to the given topology.
+ */
+ public long getAssignedWorkerCount(String topologyId) {
+ return usedMemory
+ .values()
+ .stream()
+ .filter((topoAndMem) -> topologyId.equals(topoAndMem.topoId))
+ .count();
+ }
+
+ /**
+ * Clears the topology assignment and tracked memory for the given port.
+ *
+ * @param port The worker port
+ */
+ public void remove(int port) {
+ usedMemory.remove(port);
+ reservedMemory.remove(port);
+ }
+
+ /**
+ * Assigns the given topology id to the given port, and sets the used memory for that port and topology id.
+ *
+ * @param port The worker port
+ * @param topologyId The topology id
+ * @param usedMemoryMb The memory used by the topology
+ */
+ public void setUsedMemoryMb(int port, String topologyId, long usedMemoryMb) {
+ usedMemory.put(port, new TopoAndMemory(topologyId, usedMemoryMb));
+ }
+
+ /**
+ * Sets the reserved memory for the given port and topology id.
+ *
+ * @param port The worker port
+ * @param topologyId The topology id
+ * @param reservedMemoryMb The memory reserved by the topology
+ */
+ public void setReservedMemoryMb(int port, String topologyId, long reservedMemoryMb) {
+ reservedMemory.put(port, new TopoAndMemory(topologyId, reservedMemoryMb));
+ }
+
+ private static class TopoAndMemory {
+
+ public final String topoId;
+ public final long memory;
+
+ public TopoAndMemory(String id, long mem) {
+ topoId = id;
+ memory = mem;
+ }
+
+ @Override
+ public String toString() {
+ return "{TOPO: " + topoId + " at " + memory + " MB}";
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java
index e972feb..1a5fd82 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainer.java
@@ -19,6 +19,7 @@ import org.apache.storm.daemon.worker.Worker;
import org.apache.storm.generated.LocalAssignment;
import org.apache.storm.generated.ProfileRequest;
import org.apache.storm.messaging.IContext;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -29,8 +30,10 @@ public class LocalContainer extends Container {
private volatile boolean _isAlive = false;
public LocalContainer(Map<String, Object> conf, String supervisorId, int supervisorPort, int port,
- LocalAssignment assignment, IContext sharedContext) throws IOException {
- super(ContainerType.LAUNCH, conf, supervisorId, supervisorPort, port, assignment, null, null, null, null);
+ LocalAssignment assignment, IContext sharedContext, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
+ super(ContainerType.LAUNCH, conf, supervisorId, supervisorPort, port, assignment, null, null, null, null, metricsRegistry,
+ containerMemoryTracker);
_sharedContext = sharedContext;
_workerId = Utils.uuid();
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java
index c2ff66f..77b04d4 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/LocalContainerLauncher.java
@@ -16,6 +16,7 @@ import java.io.IOException;
import java.util.Map;
import org.apache.storm.generated.LocalAssignment;
import org.apache.storm.messaging.IContext;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.LocalState;
/**
@@ -26,18 +27,24 @@ public class LocalContainerLauncher extends ContainerLauncher {
private final String _supervisorId;
private final int _supervisorPort;
private final IContext _sharedContext;
+ private final StormMetricsRegistry metricsRegistry;
+ private final ContainerMemoryTracker containerMemoryTracker;
public LocalContainerLauncher(Map<String, Object> conf, String supervisorId, int supervisorPort,
- IContext sharedContext) {
+ IContext sharedContext, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) {
_conf = conf;
_supervisorId = supervisorId;
_supervisorPort = supervisorPort;
_sharedContext = sharedContext;
+ this.metricsRegistry = metricsRegistry;
+ this.containerMemoryTracker = containerMemoryTracker;
}
@Override
public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
- LocalContainer ret = new LocalContainer(_conf, _supervisorId, _supervisorPort, port, assignment, _sharedContext);
+ LocalContainer ret = new LocalContainer(_conf, _supervisorId, _supervisorPort,
+ port, assignment, _sharedContext, metricsRegistry, containerMemoryTracker);
ret.setup();
ret.launch();
return ret;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java
index c948e31..ff75858 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java
@@ -68,6 +68,7 @@ public class ReadClusterState implements Runnable, AutoCloseable {
private final LocalState localState;
private final AtomicReference<Map<Long, LocalAssignment>> cachedAssignments;
private final OnlyLatestExecutor<Integer> metricsExec;
+ private final SlotMetrics slotMetrics;
private WorkerMetricsProcessor metricsProcessor;
public ReadClusterState(Supervisor supervisor) throws Exception {
@@ -81,8 +82,10 @@ public class ReadClusterState implements Runnable, AutoCloseable {
this.localState = supervisor.getLocalState();
this.cachedAssignments = supervisor.getCurrAssignment();
this.metricsExec = new OnlyLatestExecutor<>(supervisor.getHeartbeatExecutor());
+ this.slotMetrics = supervisor.getSlotMetrics();
- this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisorPort, supervisor.getSharedContext());
+ this.launcher = ContainerLauncher.make(superConf, assignmentId, supervisorPort,
+ supervisor.getSharedContext(), supervisor.getMetricsRegistry(), supervisor.getContainerMemoryTracker());
this.metricsProcessor = null;
try {
@@ -121,7 +124,7 @@ public class ReadClusterState implements Runnable, AutoCloseable {
private Slot mkSlot(int port) throws Exception {
return new Slot(localizer, superConf, launcher, host, port,
- localState, stormClusterState, iSuper, cachedAssignments, metricsExec, metricsProcessor);
+ localState, stormClusterState, iSuper, cachedAssignments, metricsExec, metricsProcessor, slotMetrics);
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java
index 8b134b5..dae4826 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainer.java
@@ -25,6 +25,7 @@ import java.util.List;
import java.util.Map;
import org.apache.storm.container.ResourceIsolationInterface;
import org.apache.storm.generated.LocalAssignment;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.LocalState;
import org.apache.storm.utils.ServerUtils;
import org.apache.storm.utils.Utils;
@@ -37,17 +38,19 @@ public class RunAsUserContainer extends BasicContainer {
public RunAsUserContainer(Container.ContainerType type, Map<String, Object> conf, String supervisorId,
int supervisorPort, int port, LocalAssignment assignment,
ResourceIsolationInterface resourceIsolationManager, LocalState localState,
- String workerId) throws IOException {
- this(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, workerId,
- null, null, null);
+ String workerId, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
+ this(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState, workerId, metricsRegistry,
+ containerMemoryTracker, null, null, null);
}
RunAsUserContainer(Container.ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort,
int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- LocalState localState, String workerId, Map<String, Object> topoConf, AdvancedFSOps ops,
- String profileCmd) throws IOException {
+ LocalState localState, String workerId, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker, Map<String, Object> topoConf,
+ AdvancedFSOps ops, String profileCmd) throws IOException {
super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState,
- workerId, topoConf, ops, profileCmd);
+ workerId, metricsRegistry, containerMemoryTracker, topoConf, ops, profileCmd);
if (Utils.isOnWindows()) {
throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java
index c0bb47f..2e7aa9c 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/RunAsUserContainerLauncher.java
@@ -17,6 +17,7 @@ import java.util.Map;
import org.apache.storm.container.ResourceIsolationInterface;
import org.apache.storm.daemon.supervisor.Container.ContainerType;
import org.apache.storm.generated.LocalAssignment;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.LocalState;
public class RunAsUserContainerLauncher extends ContainerLauncher {
@@ -24,19 +25,24 @@ public class RunAsUserContainerLauncher extends ContainerLauncher {
private final Map<String, Object> _conf;
private final String _supervisorId;
private final int _supervisorPort;
+ private final StormMetricsRegistry metricsRegistry;
+ private final ContainerMemoryTracker containerMemoryTracker;
public RunAsUserContainerLauncher(Map<String, Object> conf, String supervisorId, int supervisorPort,
- ResourceIsolationInterface resourceIsolationManager) throws IOException {
+ ResourceIsolationInterface resourceIsolationManager, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
_conf = conf;
_supervisorId = supervisorId;
_supervisorPort = supervisorPort;
_resourceIsolationManager = resourceIsolationManager;
+ this.metricsRegistry = metricsRegistry;
+ this.containerMemoryTracker = containerMemoryTracker;
}
@Override
public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
Container container = new RunAsUserContainer(ContainerType.LAUNCH, _conf, _supervisorId, _supervisorPort, port,
- assignment, _resourceIsolationManager, state, null, null, null, null);
+ assignment, _resourceIsolationManager, state, null, metricsRegistry, containerMemoryTracker, null, null, null);
container.setup();
container.launch();
return container;
@@ -45,13 +51,15 @@ public class RunAsUserContainerLauncher extends ContainerLauncher {
@Override
public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
return new RunAsUserContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, _supervisorPort, port,
- assignment, _resourceIsolationManager, state, null, null, null, null);
+ assignment, _resourceIsolationManager, state, null, metricsRegistry, containerMemoryTracker,
+ null, null, null);
}
@Override
public Killable recoverContainer(String workerId, LocalState localState) throws IOException {
return new RunAsUserContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, _supervisorPort, -1, null,
- _resourceIsolationManager, localState, workerId, null, null, null);
+ _resourceIsolationManager, localState, workerId, metricsRegistry, containerMemoryTracker,
+ null, null, null);
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
index 4e87c77..e27f8f4 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java
@@ -56,10 +56,8 @@ import org.slf4j.LoggerFactory;
public class Slot extends Thread implements AutoCloseable, BlobChangingCallback {
private static final Logger LOG = LoggerFactory.getLogger(Slot.class);
- private static final Meter numWorkersLaunched =
- StormMetricsRegistry.registerMeter("supervisor:num-workers-launched");
- private enum KillReason {
+ enum KillReason {
ASSIGNMENT_CHANGED, BLOB_CHANGED, PROCESS_EXIT, MEMORY_VIOLATION, HB_TIMEOUT, HB_NULL;
@Override
@@ -69,11 +67,6 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
}
- private static final Map<KillReason, Meter> numWorkersKilledFor = EnumUtil.toEnumMap(KillReason.class,
- killReason -> StormMetricsRegistry.registerMeter("supervisor:num-workers-killed-" + killReason.toString()));
- private static final Timer workerLaunchDuration = StormMetricsRegistry.registerTimer(
- "supervisor:worker-launch-duration");
-
private static final long ONE_SEC_IN_NANO = TimeUnit.NANOSECONDS.convert(1, TimeUnit.SECONDS);
private final AtomicReference<LocalAssignment> newAssignment = new AtomicReference<>();
private final AtomicReference<Set<TopoProfileAction>> profiling = new AtomicReference<>(new HashSet<>());
@@ -93,7 +86,8 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
ISupervisor iSupervisor,
AtomicReference<Map<Long, LocalAssignment>> cachedCurrentAssignments,
OnlyLatestExecutor<Integer> metricsExec,
- WorkerMetricsProcessor metricsProcessor) throws Exception {
+ WorkerMetricsProcessor metricsProcessor,
+ SlotMetrics slotMetrics) throws Exception {
super("SLOT_" + port);
this.metricsExec = metricsExec;
this.cachedCurrentAssignments = cachedCurrentAssignments;
@@ -109,7 +103,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
iSupervisor,
localState,
this,
- metricsExec, metricsProcessor);
+ metricsExec, metricsProcessor, slotMetrics);
LocalAssignment currentAssignment = null;
Container container = null;
@@ -148,7 +142,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
//if the current assignment is already running, new assignment will never be promoted to currAssignment,
// because Timer is not being compared in #equals or #equivalent, meaning newAssignment always equals to currAssignment.
// Therefore the timer in newAssignment won't be invoked
- this.dynamicState = new DynamicState(currentAssignment, container, this.newAssignment.get());
+ this.dynamicState = new DynamicState(currentAssignment, container, this.newAssignment.get(), slotMetrics);
if (MachineState.RUNNING == dynamicState.state) {
//We are running so we should recover the blobs.
staticState.localizer.recoverRunningTopology(currentAssignment, port, this);
@@ -272,7 +266,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
}
dynamicState.container.kill();
}
- numWorkersKilledFor.get(reason).mark();
+ staticState.slotMetrics.numWorkersKilledFor.get(reason).mark();
DynamicState next;
switch (reason) {
@@ -456,7 +450,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
}
dynamicState = updateAssignmentIfNeeded(dynamicState);
- numWorkersLaunched.mark();
+ staticState.slotMetrics.numWorkersLaunched.mark();
Container c =
staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState);
return dynamicState
@@ -811,7 +805,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
* @param newAssignment the new assignment for this slot to run, null to run nothing
*/
public void setNewAssignment(LocalAssignment newAssignment) {
- this.newAssignment.set(newAssignment == null ? null : new TimerDecoratedAssignment(newAssignment, workerLaunchDuration));
+ this.newAssignment.set(newAssignment == null ? null : new TimerDecoratedAssignment(newAssignment, staticState.slotMetrics.workerLaunchDuration));
}
@Override
@@ -878,6 +872,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
} while (!cachedCurrentAssignments.compareAndSet(orig, update));
}
+ @Override
public void run() {
try {
while (!done) {
@@ -991,6 +986,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
public final BlobChangingCallback changingCallback;
public final OnlyLatestExecutor<Integer> metricsExec;
public final WorkerMetricsProcessor metricsProcessor;
+ public final SlotMetrics slotMetrics;
StaticState(AsyncLocalizer localizer, long hbTimeoutMs, long firstHbTimeoutMs,
long killSleepMs, long monitorFreqMs,
@@ -998,7 +994,8 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
ISupervisor iSupervisor, LocalState localState,
BlobChangingCallback changingCallback,
OnlyLatestExecutor<Integer> metricsExec,
- WorkerMetricsProcessor metricsProcessor) {
+ WorkerMetricsProcessor metricsProcessor,
+ SlotMetrics slotMetrics) {
this.localizer = localizer;
this.hbTimeoutMs = hbTimeoutMs;
this.firstHbTimeoutMs = firstHbTimeoutMs;
@@ -1012,17 +1009,11 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.changingCallback = changingCallback;
this.metricsExec = metricsExec;
this.metricsProcessor = metricsProcessor;
+ this.slotMetrics = slotMetrics;
}
}
static class DynamicState {
- private static final Map<MachineState, Meter> transitionIntoState = EnumUtil.toEnumMap(MachineState.class,
- machineState -> StormMetricsRegistry.registerMeter("supervisor:num-worker-transitions-into-" + machineState.toString()));
- //This also tracks how many times worker transitioning out of a state
- private static final Map<MachineState, Timer> timeSpentInState = EnumUtil.toEnumMap(MachineState.class,
- machineState -> StormMetricsRegistry.registerTimer("supervisor:time-worker-spent-in-state-" + machineState.toString() + "-ms")
- );
-
public final MachineState state;
/**
@@ -1066,8 +1057,10 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
* The entering time when the machine transitions to current state.
*/
public final long startTime;
+ private final SlotMetrics slotMetrics;
- public DynamicState(final LocalAssignment currentAssignment, Container container, final LocalAssignment newAssignment) {
+ public DynamicState(final LocalAssignment currentAssignment, Container container, final LocalAssignment newAssignment,
+ SlotMetrics slotMetrics) {
this.currentAssignment = currentAssignment;
this.container = container;
if ((currentAssignment == null) ^ (container == null)) {
@@ -1079,7 +1072,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
} else {
state = MachineState.RUNNING;
}
- transitionIntoState.get(state).mark();
+ slotMetrics.transitionIntoState.get(state).mark();
this.startTime = Time.currentTimeMillis();
this.newAssignment = newAssignment;
@@ -1090,6 +1083,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.changingBlobs = Collections.emptySet();
this.pendingChangingBlobsAssignment = null;
this.pendingChangingBlobs = Collections.emptySet();
+ this.slotMetrics = slotMetrics;
}
public DynamicState(final MachineState state, final LocalAssignment newAssignment,
@@ -1098,7 +1092,8 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
final Future<Void> pendingDownload, final Set<TopoProfileAction> profileActions,
final Set<TopoProfileAction> pendingStopProfileActions,
final Set<BlobChanging> changingBlobs,
- final Set<Future<Void>> pendingChangingBlobs, final LocalAssignment pendingChaningBlobsAssignment) {
+ final Set<Future<Void>> pendingChangingBlobs, final LocalAssignment pendingChaningBlobsAssignment,
+ final SlotMetrics slotMetrics) {
assert pendingChangingBlobs != null;
assert pendingChangingBlobs.isEmpty() == (pendingChaningBlobsAssignment == null);
this.state = state;
@@ -1113,6 +1108,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.changingBlobs = changingBlobs;
this.pendingChangingBlobs = pendingChangingBlobs;
this.pendingChangingBlobsAssignment = pendingChaningBlobsAssignment;
+ this.slotMetrics = slotMetrics;
}
public String toString() {
@@ -1139,7 +1135,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingLocalization, this.startTime,
this.pendingDownload, this.profileActions,
this.pendingStopProfileActions, this.changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withPendingLocalization(LocalAssignment pendingLocalization, Future<Void> pendingDownload) {
@@ -1148,7 +1144,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
pendingLocalization, this.startTime,
pendingDownload, this.profileActions,
this.pendingStopProfileActions, this.changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withPendingLocalization(Future<Void> pendingDownload) {
@@ -1164,8 +1160,8 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
public DynamicState withState(final MachineState state) {
long newStartTime = Time.currentTimeMillis();
//We may (though unlikely) lose metering here if state transition is too frequent (less than a millisecond)
- timeSpentInState.get(this.state).update(newStartTime - startTime, TimeUnit.MILLISECONDS);
- transitionIntoState.get(state).mark();
+ slotMetrics.timeSpentInState.get(this.state).update(newStartTime - startTime, TimeUnit.MILLISECONDS);
+ slotMetrics.transitionIntoState.get(state).mark();
LocalAssignment assignment = this.currentAssignment;
if (MachineState.RUNNING != this.state && MachineState.RUNNING == state
@@ -1180,7 +1176,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingLocalization, newStartTime,
this.pendingDownload, this.profileActions,
this.pendingStopProfileActions, this.changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withCurrentAssignment(final Container container, final LocalAssignment currentAssignment) {
@@ -1189,7 +1185,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingLocalization, this.startTime,
this.pendingDownload, this.profileActions,
this.pendingStopProfileActions, this.changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withProfileActions(Set<TopoProfileAction> profileActions, Set<TopoProfileAction> pendingStopProfileActions) {
@@ -1198,7 +1194,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingLocalization, this.startTime,
this.pendingDownload, profileActions,
pendingStopProfileActions, this.changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withChangingBlobs(Set<BlobChanging> changingBlobs) {
@@ -1210,7 +1206,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingLocalization, this.startTime,
this.pendingDownload, profileActions,
this.pendingStopProfileActions, changingBlobs,
- this.pendingChangingBlobs, this.pendingChangingBlobsAssignment);
+ this.pendingChangingBlobs, this.pendingChangingBlobsAssignment, this.slotMetrics);
}
public DynamicState withPendingChangingBlobs(Set<Future<Void>> pendingChangingBlobs,
@@ -1221,7 +1217,7 @@ public class Slot extends Thread implements AutoCloseable, BlobChangingCallback
this.pendingDownload, profileActions,
this.pendingStopProfileActions, this.changingBlobs,
pendingChangingBlobs,
- pendingChangingBlobsAssignment);
+ pendingChangingBlobsAssignment, this.slotMetrics);
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SlotMetrics.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SlotMetrics.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SlotMetrics.java
new file mode 100644
index 0000000..f8e13fd
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/SlotMetrics.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2018 The Apache Software Foundation.
+ *
+ * Licensed 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.storm.daemon.supervisor;
+
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Timer;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.utils.EnumUtil;
+
+class SlotMetrics {
+
+ final Meter numWorkersLaunched;
+ final Map<Slot.KillReason, Meter> numWorkersKilledFor;
+ final Timer workerLaunchDuration;
+ final Map<Slot.MachineState, Meter> transitionIntoState;
+ //This also tracks how many times worker transitioning out of a state
+ final Map<Slot.MachineState, Timer> timeSpentInState;
+
+ SlotMetrics(StormMetricsRegistry metricsRegistry) {
+ numWorkersLaunched = metricsRegistry.registerMeter("supervisor:num-workers-launched");
+ numWorkersKilledFor = Collections.unmodifiableMap(EnumUtil.toEnumMap(Slot.KillReason.class,
+ killReason -> metricsRegistry.registerMeter("supervisor:num-workers-killed-" + killReason.toString())));
+ workerLaunchDuration = metricsRegistry.registerTimer("supervisor:worker-launch-duration");
+ transitionIntoState = Collections.unmodifiableMap(EnumUtil.toEnumMap(Slot.MachineState.class,
+ machineState -> metricsRegistry.registerMeter("supervisor:num-worker-transitions-into-" + machineState.toString())));
+ timeSpentInState = Collections.unmodifiableMap(EnumUtil.toEnumMap(Slot.MachineState.class,
+ machineState -> metricsRegistry.registerTimer("supervisor:time-worker-spent-in-state-" + machineState.toString() + "-ms")));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java
index 25ddf15..d6e5744 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java
@@ -102,6 +102,9 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
// to really make this work well.
private final ExecutorService heartbeatExecutor;
private final AsyncLocalizer asyncLocalizer;
+ private final StormMetricsRegistry metricsRegistry;
+ private final ContainerMemoryTracker containerMemoryTracker;
+ private final SlotMetrics slotMetrics;
private volatile boolean active;
private EventManager eventManager;
private ReadClusterState readState;
@@ -109,9 +112,9 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
//used for local cluster heartbeating
private Nimbus.Iface localNimbus;
- private Supervisor(ISupervisor iSupervisor)
+ private Supervisor(ISupervisor iSupervisor, StormMetricsRegistry metricsRegistry)
throws IOException, IllegalAccessException, InstantiationException, ClassNotFoundException {
- this(ConfigUtils.readStormConfig(), null, iSupervisor);
+ this(ConfigUtils.readStormConfig(), null, iSupervisor, metricsRegistry);
}
/**
@@ -122,9 +125,12 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
* @param iSupervisor {@link ISupervisor}
* @throws IOException
*/
- public Supervisor(Map<String, Object> conf, IContext sharedContext, ISupervisor iSupervisor)
+ public Supervisor(Map<String, Object> conf, IContext sharedContext, ISupervisor iSupervisor, StormMetricsRegistry metricsRegistry)
throws IOException, IllegalAccessException, ClassNotFoundException, InstantiationException {
this.conf = conf;
+ this.metricsRegistry = metricsRegistry;
+ this.containerMemoryTracker = new ContainerMemoryTracker(metricsRegistry);
+ this.slotMetrics = new SlotMetrics(metricsRegistry);
this.iSupervisor = iSupervisor;
this.active = true;
this.upTime = Utils.makeUptimeComputer();
@@ -152,7 +158,7 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
try {
this.localState = ServerConfigUtils.supervisorState(conf);
- this.asyncLocalizer = new AsyncLocalizer(conf);
+ this.asyncLocalizer = new AsyncLocalizer(conf, metricsRegistry);
} catch (IOException e) {
throw Utils.wrapInRuntime(e);
}
@@ -180,8 +186,9 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
*/
public static void main(String[] args) throws Exception {
Utils.setupDefaultUncaughtExceptionHandler();
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
@SuppressWarnings("resource")
- Supervisor instance = new Supervisor(new StandaloneSupervisor());
+ Supervisor instance = new Supervisor(new StandaloneSupervisor(), metricsRegistry);
instance.launchDaemon();
}
@@ -200,6 +207,18 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
return sharedContext;
}
+ StormMetricsRegistry getMetricsRegistry() {
+ return metricsRegistry;
+ }
+
+ ContainerMemoryTracker getContainerMemoryTracker() {
+ return containerMemoryTracker;
+ }
+
+ SlotMetrics getSlotMetrics() {
+ return slotMetrics;
+ }
+
public Map<String, Object> getConf() {
return conf;
}
@@ -310,13 +329,16 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
throw new IllegalArgumentException("Cannot start server in local mode!");
}
launch();
- Utils.addShutdownHookWithForceKillIn1Sec(this::close);
- StormMetricsRegistry.registerGauge("supervisor:num-slots-used-gauge", () -> SupervisorUtils.supervisorWorkerIds(conf).size());
+ metricsRegistry.registerGauge("supervisor:num-slots-used-gauge", () -> SupervisorUtils.supervisorWorkerIds(conf).size());
//This will only get updated once
- StormMetricsRegistry.registerMeter("supervisor:num-launched").mark();
- StormMetricsRegistry.registerMeter("supervisor:num-shell-exceptions", ShellUtils.numShellExceptions);
- StormMetricsRegistry.startMetricsReporters(conf);
+ metricsRegistry.registerMeter("supervisor:num-launched").mark();
+ metricsRegistry.registerMeter("supervisor:num-shell-exceptions", ShellUtils.numShellExceptions);
+ metricsRegistry.startMetricsReporters(conf);
+ Utils.addShutdownHookWithForceKillIn1Sec(() -> {
+ metricsRegistry.stopMetricsReporters();
+ this.close();
+ });
// blocking call under the hood, must invoke after launch cause some services must be initialized
launchSupervisorThriftServer(conf);
@@ -513,7 +535,7 @@ public class Supervisor implements DaemonCommon, AutoCloseable {
} else {
try {
ContainerLauncher launcher = ContainerLauncher.make(getConf(), getId(), getThriftServerPort(),
- getSharedContext());
+ getSharedContext(), getMetricsRegistry(), getContainerMemoryTracker());
killWorkers(SupervisorUtils.supervisorWorkerIds(conf), launcher);
} catch (Exception e) {
throw Utils.wrapInRuntime(e);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java
index 29d4f84..28bb2dd 100644
--- a/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java
+++ b/storm-server/src/main/java/org/apache/storm/localizer/AsyncLocalizer.java
@@ -73,13 +73,10 @@ public class AsyncLocalizer implements AutoCloseable {
private static final CompletableFuture<Void> ALL_DONE_FUTURE = new CompletableFuture<>();
private static final int ATTEMPTS_INTERVAL_TIME = 100;
- private static final Timer singleBlobLocalizationDuration = StormMetricsRegistry.registerTimer(
- "supervisor:single-blob-localization-duration");
- private static final Timer blobCacheUpdateDuration = StormMetricsRegistry.registerTimer("supervisor:blob-cache-update-duration");
- private static final Timer blobLocalizationDuration = StormMetricsRegistry.registerTimer("supervisor:blob-localization-duration");
-
- private static final Meter numBlobUpdateVersionChanged = StormMetricsRegistry.registerMeter(
- "supervisor:num-blob-update-version-changed");
+ private final Timer singleBlobLocalizationDuration;
+ private final Timer blobCacheUpdateDuration;
+ private final Timer blobLocalizationDuration;
+ private final Meter numBlobUpdateVersionChanged;
static {
ALL_DONE_FUTURE.complete(null);
@@ -103,14 +100,19 @@ public class AsyncLocalizer implements AutoCloseable {
private final int blobDownloadRetries;
private final ScheduledExecutorService execService;
private final long cacheCleanupPeriod;
+ private final StormMetricsRegistry metricsRegistry;
// cleanup
@VisibleForTesting
protected long cacheTargetSize;
@VisibleForTesting
- AsyncLocalizer(Map<String, Object> conf, AdvancedFSOps ops, String baseDir) throws IOException {
-
+ AsyncLocalizer(Map<String, Object> conf, AdvancedFSOps ops, String baseDir, StormMetricsRegistry metricsRegistry) throws IOException {
this.conf = conf;
+ this.singleBlobLocalizationDuration = metricsRegistry.registerTimer("supervisor:single-blob-localization-duration");
+ this.blobCacheUpdateDuration = metricsRegistry.registerTimer("supervisor:blob-cache-update-duration");
+ this.blobLocalizationDuration = metricsRegistry.registerTimer("supervisor:blob-localization-duration");
+ this.numBlobUpdateVersionChanged = metricsRegistry.registerMeter("supervisor:num-blob-update-version-changed");
+ this.metricsRegistry = metricsRegistry;
isLocalMode = ConfigUtils.isLocalMode(conf);
fsOps = ops;
localBaseDir = Paths.get(baseDir);
@@ -134,8 +136,8 @@ public class AsyncLocalizer implements AutoCloseable {
blobPending = new ConcurrentHashMap<>();
}
- public AsyncLocalizer(Map<String, Object> conf) throws IOException {
- this(conf, AdvancedFSOps.make(conf), ConfigUtils.supervisorLocalDir(conf));
+ public AsyncLocalizer(Map<String, Object> conf, StormMetricsRegistry metricsRegistry) throws IOException {
+ this(conf, AdvancedFSOps.make(conf), ConfigUtils.supervisorLocalDir(conf), metricsRegistry);
}
@VisibleForTesting
@@ -145,7 +147,7 @@ public class AsyncLocalizer implements AutoCloseable {
try {
return new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps,
LocallyCachedTopologyBlob.TopologyBlobType
- .TOPO_JAR, owner);
+ .TOPO_JAR, owner, metricsRegistry);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -159,7 +161,7 @@ public class AsyncLocalizer implements AutoCloseable {
try {
return new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps,
LocallyCachedTopologyBlob.TopologyBlobType
- .TOPO_CODE, owner);
+ .TOPO_CODE, owner, metricsRegistry);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -173,7 +175,7 @@ public class AsyncLocalizer implements AutoCloseable {
try {
return new LocallyCachedTopologyBlob(topologyId, isLocalMode, conf, fsOps,
LocallyCachedTopologyBlob.TopologyBlobType
- .TOPO_CONF, owner);
+ .TOPO_CONF, owner, metricsRegistry);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -183,13 +185,15 @@ public class AsyncLocalizer implements AutoCloseable {
private LocalizedResource getUserArchive(String user, String key) {
assert user != null : "All user archives require a user present";
ConcurrentMap<String, LocalizedResource> keyToResource = userArchives.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
- return keyToResource.computeIfAbsent(key, (k) -> new LocalizedResource(key, localBaseDir, true, fsOps, conf, user));
+ return keyToResource.computeIfAbsent(key,
+ (k) -> new LocalizedResource(key, localBaseDir, true, fsOps, conf, user, metricsRegistry));
}
private LocalizedResource getUserFile(String user, String key) {
assert user != null : "All user archives require a user present";
ConcurrentMap<String, LocalizedResource> keyToResource = userFiles.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
- return keyToResource.computeIfAbsent(key, (k) -> new LocalizedResource(key, localBaseDir, false, fsOps, conf, user));
+ return keyToResource.computeIfAbsent(key,
+ (k) -> new LocalizedResource(key, localBaseDir, false, fsOps, conf, user, metricsRegistry));
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java
index 6f1c8d7..541c075 100644
--- a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java
+++ b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResource.java
@@ -49,6 +49,7 @@ import org.apache.storm.daemon.supervisor.IAdvancedFSOps;
import org.apache.storm.generated.AuthorizationException;
import org.apache.storm.generated.KeyNotFoundException;
import org.apache.storm.generated.ReadableBlobMeta;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.ObjectReader;
@@ -92,8 +93,8 @@ public class LocalizedResource extends LocallyCachedBlob {
private long size = -1;
LocalizedResource(String key, Path localBaseDir, boolean shouldUncompress, IAdvancedFSOps fsOps, Map<String, Object> conf,
- String user) {
- super(key + (shouldUncompress ? " archive" : " file"), key);
+ String user, StormMetricsRegistry metricRegistry) {
+ super(key + (shouldUncompress ? " archive" : " file"), key, metricRegistry);
Path base = getLocalUserFileCacheDir(localBaseDir, user);
this.baseDir = shouldUncompress ? getCacheDirForArchives(base) : getCacheDirForFiles(base);
this.conf = conf;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java
index f12713b..a117032 100644
--- a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java
+++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedBlob.java
@@ -12,9 +12,7 @@
package org.apache.storm.localizer;
-import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Histogram;
-import com.codahale.metrics.Timer;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
@@ -52,7 +50,7 @@ public abstract class LocallyCachedBlob {
private long lastUsed = Time.currentTimeMillis();
private CompletableFuture<Void> doneUpdating = null;
- private static final Histogram fetchingRate = StormMetricsRegistry.registerHistogram("supervisor:blob-fetching-rate-MB/s");
+ private final Histogram fetchingRate;
/**
* Create a new LocallyCachedBlob.
@@ -60,9 +58,10 @@ public abstract class LocallyCachedBlob {
* @param blobDescription a description of the blob this represents. Typically it should at least be the blob key, but ideally also
* include if it is an archive or not, what user or topology it is for, or if it is a storm.jar etc.
*/
- protected LocallyCachedBlob(String blobDescription, String blobKey) {
+ protected LocallyCachedBlob(String blobDescription, String blobKey, StormMetricsRegistry metricsRegistry) {
this.blobDescription = blobDescription;
this.blobKey = blobKey;
+ this.fetchingRate = metricsRegistry.registerHistogram("supervisor:blob-fetching-rate-MB/s");
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
index 8224f40..795abe7 100644
--- a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
+++ b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
@@ -34,7 +34,7 @@ import org.apache.storm.blobstore.ClientBlobStore;
import org.apache.storm.daemon.supervisor.AdvancedFSOps;
import org.apache.storm.generated.AuthorizationException;
import org.apache.storm.generated.KeyNotFoundException;
-import org.apache.storm.generated.LocalAssignment;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.ServerConfigUtils;
import org.apache.storm.utils.ServerUtils;
@@ -60,13 +60,14 @@ public class LocallyCachedTopologyBlob extends LocallyCachedBlob {
/**
* Create a new LocallyCachedBlob.
- * @param topologyId the ID of the topology.
+ * @param topologyId the ID of the topology.
* @param type the type of the blob.
* @param owner the name of the user that owns this blob.
*/
protected LocallyCachedTopologyBlob(final String topologyId, final boolean isLocalMode, final Map<String, Object> conf,
- final AdvancedFSOps fsOps, final TopologyBlobType type, String owner) throws IOException {
- super(topologyId + " " + type.getFileName(), type.getKey(topologyId));
+ final AdvancedFSOps fsOps, final TopologyBlobType type,
+ String owner, StormMetricsRegistry metricsRegistry) throws IOException {
+ super(topologyId + " " + type.getFileName(), type.getKey(topologyId), metricsRegistry);
this.topologyId = topologyId;
this.type = type;
this.isLocalMode = isLocalMode;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java b/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
index ea8867e..c4d2f3f 100644
--- a/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
+++ b/storm-server/src/main/java/org/apache/storm/metric/StormMetricsRegistry.java
@@ -12,111 +12,65 @@
package org.apache.storm.metric;
-import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
-import com.codahale.metrics.Metric;
import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.MetricSet;
import com.codahale.metrics.Reservoir;
import com.codahale.metrics.Timer;
-
+import java.util.List;
import java.util.Map;
-
-import com.google.common.annotations.VisibleForTesting;
import org.apache.storm.daemon.metrics.MetricsUtils;
import org.apache.storm.daemon.metrics.reporters.PreparableReporter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class StormMetricsRegistry extends MetricRegistry {
- @VisibleForTesting
- static final StormMetricsRegistry REGISTRY = new StormMetricsRegistry();
+public class StormMetricsRegistry {
private static final Logger LOG = LoggerFactory.getLogger(StormMetricsRegistry.class);
+ private final MetricRegistry registry = new MetricRegistry();
+ private List<PreparableReporter> reporters;
+ private boolean reportersStarted = false;
- private StormMetricsRegistry() {/*Singleton pattern*/}
-
- public static <V> Gauge<V> registerGauge(final String name, final Gauge<V> gauge) {
- return REGISTRY.register(name, gauge);
+ public Meter registerMeter(String name) {
+ return registry.meter(name);
}
-
- public static Histogram registerHistogram(String name) {
- return registerHistogram(name, new ExponentiallyDecayingReservoir());
+
+ public Meter registerMeter(String name, Meter meter) {
+ return registry.meter(name, () -> meter);
}
-
- public static Histogram registerHistogram(String name, Reservoir reservoir) {
- return REGISTRY.register(name, new Histogram(reservoir));
- }
-
- public static Meter registerMeter(String name) {
- return REGISTRY.register(name, new Meter());
+
+ public Timer registerTimer(String name) {
+ return registry.timer(name);
}
- public static void registerMeter(String name, Meter meter) {
- REGISTRY.register(name, meter);
+ public Histogram registerHistogram(String name) {
+ return registry.histogram(name);
}
- public static void registerMetricSet(MetricSet metrics) {
- REGISTRY.registerAll(metrics);
- }
-
- public static void unregisterMetricSet(MetricSet metrics) {
- unregisterMetricSet(null, metrics);
- }
-
- public static void unregisterMetricSet(String prefix, MetricSet metrics) {
- for (Map.Entry<String, Metric> entry : metrics.getMetrics().entrySet()) {
- final String name = name(prefix, entry.getKey());
- if (entry.getValue() instanceof MetricSet) {
- unregisterMetricSet(name, (MetricSet) entry.getValue());
- } else {
- REGISTRY.remove(name);
- }
- }
+ public Histogram registerHistogram(String name, Reservoir reservoir) {
+ return registry.histogram(name, () -> new Histogram(reservoir));
}
- public static Timer registerTimer(String name) {
- return REGISTRY.register(name, new Timer());
+ public <V> Gauge<V> registerGauge(final String name, Gauge<V> gauge) {
+ return registry.gauge(name, () -> gauge);
}
- /**
- * Start metrics reporters for the registry singleton.
- *
- * @param topoConf config that specifies reporter plugin
- */
- public static void startMetricsReporters(Map<String, Object> topoConf) {
- for (PreparableReporter reporter : MetricsUtils.getPreparableReporters(topoConf)) {
- reporter.prepare(StormMetricsRegistry.REGISTRY, topoConf);
+ public void startMetricsReporters(Map<String, Object> daemonConf) {
+ reporters = MetricsUtils.getPreparableReporters(daemonConf);
+ for (PreparableReporter reporter : reporters) {
+ reporter.prepare(registry, daemonConf);
reporter.start();
LOG.info("Started statistics report plugin...");
}
+ reportersStarted = true;
}
- /**
- * Override parent method to swallow exceptions for double registration, including MetricSet registration
- * This is more similar to super#getOrAdd than super#register.
- * Notice that this method is only accessible to the private singleton, hence private to client code.
- *
- * @param name name of the metric
- * @param metric metric to be registered
- * @param <T> type of metric
- * @return metric just registered or existing metric, if double registration occurs.
- * @throws IllegalArgumentException name already exist with a different kind of metric
- */
- @Override
- public <T extends Metric> T register(final String name, T metric) throws IllegalArgumentException {
- assert !(metric instanceof MetricSet);
- try {
- return super.register(name, metric);
- } catch (IllegalArgumentException e) {
- @SuppressWarnings("unchecked")
- final T existing = (T) REGISTRY.getMetrics().get(name);
- if (metric.getClass().isInstance(existing)) {
- LOG.warn("Metric {} has already been registered", name);
- return existing;
+ public void stopMetricsReporters() {
+ if (reportersStarted) {
+ for (PreparableReporter reporter : reporters) {
+ reporter.stop();
}
- throw e;
+ reportersStarted = false;
}
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
index 71f53a5..bf2b840 100644
--- a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStore.java
@@ -12,6 +12,7 @@
package org.apache.storm.metricstore;
import java.util.Map;
+import org.apache.storm.metric.StormMetricsRegistry;
public interface MetricStore extends AutoCloseable {
@@ -19,9 +20,10 @@ public interface MetricStore extends AutoCloseable {
* Create metric store instance using the configurations provided via the config map.
*
* @param config Storm config map
+ * @param metricsRegistry The Nimbus daemon metrics registry
* @throws MetricException on preparation error
*/
- void prepare(Map<String, Object> config) throws MetricException;
+ void prepare(Map<String, Object> config, StormMetricsRegistry metricsRegistry) throws MetricException;
/**
* Stores a metric in the store.
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
index 42e5a7d..c375c06 100644
--- a/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/MetricStoreConfig.java
@@ -13,6 +13,7 @@ package org.apache.storm.metricstore;
import java.util.Map;
import org.apache.storm.DaemonConfig;
+import org.apache.storm.metric.StormMetricsRegistry;
public class MetricStoreConfig {
@@ -20,15 +21,16 @@ public class MetricStoreConfig {
/**
* Configures metrics store (running on Nimbus) to use the class specified in the conf.
* @param conf Storm config map
+ * @param metricsRegistry The Nimbus daemon metrics registry
* @return MetricStore prepared store
* @throws MetricException on misconfiguration
*/
- public static MetricStore configure(Map<String, Object> conf) throws MetricException {
+ public static MetricStore configure(Map<String, Object> conf, StormMetricsRegistry metricsRegistry) throws MetricException {
try {
String storeClass = (String) conf.get(DaemonConfig.STORM_METRIC_STORE_CLASS);
MetricStore store = (MetricStore) (Class.forName(storeClass)).newInstance();
- store.prepare(conf);
+ store.prepare(conf, metricsRegistry);
return store;
} catch (Exception e) {
throw new MetricException("Failed to create metric store", e);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
index 900f8b9..3c32474 100644
--- a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/MetricsCleaner.java
@@ -23,15 +23,15 @@ import org.slf4j.LoggerFactory;
*/
public class MetricsCleaner implements Runnable, AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(MetricsCleaner.class);
- private static long DEFAULT_SLEEP_MS = 4L * 60L * 60L * 1000L;
- private RocksDbStore store;
- private long retentionHours;
+ private static final long DEFAULT_SLEEP_MS = 4L * 60L * 60L * 1000L;
+ private final RocksDbStore store;
+ private final long retentionHours;
+ private final Meter failureMeter;
private volatile boolean shutdown = false;
private long sleepMs = DEFAULT_SLEEP_MS;
- private Meter failureMeter;
private long purgeTimestamp = 0L;
- MetricsCleaner(RocksDbStore store, int retentionHours, int hourlyPeriod, Meter failureMeter) {
+ MetricsCleaner(RocksDbStore store, int retentionHours, int hourlyPeriod, Meter failureMeter, StormMetricsRegistry metricsRegistry) {
this.store = store;
this.retentionHours = retentionHours;
if (hourlyPeriod > 0) {
@@ -39,7 +39,7 @@ public class MetricsCleaner implements Runnable, AutoCloseable {
}
this.failureMeter = failureMeter;
- StormMetricsRegistry.registerGauge("MetricsCleaner:purgeTimestamp", () -> purgeTimestamp);
+ metricsRegistry.registerGauge("MetricsCleaner:purgeTimestamp", () -> purgeTimestamp);
}
@Override
[3/6] storm git commit: STORM-3197: Make StormMetricsRegistry
non-static
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
index ee7f621..2ac1c20 100644
--- a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
+++ b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
@@ -55,12 +55,14 @@ public class RocksDbStore implements MetricStore, AutoCloseable {
* Create metric store instance using the configurations provided via the config map.
*
* @param config Storm config map
+ * @param metricsRegistry The Nimbus daemon metrics registry
* @throws MetricException on preparation error
*/
- public void prepare(Map<String, Object> config) throws MetricException {
+ @Override
+ public void prepare(Map<String, Object> config, StormMetricsRegistry metricsRegistry) throws MetricException {
validateConfig(config);
- this.failureMeter = StormMetricsRegistry.registerMeter("RocksDB:metric-failures");
+ this.failureMeter = metricsRegistry.registerMeter("RocksDB:metric-failures");
RocksDB.loadLibrary();
boolean createIfMissing = ObjectReader.getBoolean(config.get(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING), false);
@@ -87,7 +89,7 @@ public class RocksDbStore implements MetricStore, AutoCloseable {
if (config.containsKey(DaemonConfig.STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS)) {
deletionPeriod = Integer.parseInt(config.get(DaemonConfig.STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS).toString());
}
- metricsCleaner = new MetricsCleaner(this, retentionHours, deletionPeriod, failureMeter);
+ metricsCleaner = new MetricsCleaner(this, retentionHours, deletionPeriod, failureMeter, metricsRegistry);
// create thread to process insertion of all metrics
metricsWriter = new RocksDbMetricsWriter(this, this.queue, this.failureMeter);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java b/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java
index 3783fdb..edd7444 100644
--- a/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java
+++ b/storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java
@@ -48,12 +48,14 @@ import org.slf4j.LoggerFactory;
* A callback function when nimbus gains leadership.
*/
public class LeaderListenerCallback {
- private static final Meter numGainedLeader = StormMetricsRegistry.registerMeter("nimbus:num-gained-leadership");
- private static final Meter numLostLeader = StormMetricsRegistry.registerMeter("nimbus:num-lost-leadership");
private static final Logger LOG = LoggerFactory.getLogger(LeaderListenerCallback.class);
private static final String STORM_JAR_SUFFIX = "-stormjar.jar";
private static final String STORM_CODE_SUFFIX = "-stormcode.ser";
private static final String STORM_CONF_SUFFIX = "-stormconf.ser";
+
+ private final Meter numGainedLeader;
+ private final Meter numLostLeader;
+
private final BlobStore blobStore;
private final TopoCache tc;
private final IStormClusterState clusterState;
@@ -73,7 +75,7 @@ public class LeaderListenerCallback {
* @param acls zookeeper acls
*/
public LeaderListenerCallback(Map conf, CuratorFramework zk, LeaderLatch leaderLatch, BlobStore blobStore,
- TopoCache tc, IStormClusterState clusterState, List<ACL> acls) {
+ TopoCache tc, IStormClusterState clusterState, List<ACL> acls, StormMetricsRegistry metricsRegistry) {
this.blobStore = blobStore;
this.tc = tc;
this.clusterState = clusterState;
@@ -81,6 +83,8 @@ public class LeaderListenerCallback {
this.leaderLatch = leaderLatch;
this.conf = conf;
this.acls = acls;
+ this.numGainedLeader = metricsRegistry.registerMeter("nimbus:num-gained-leadership");
+ this.numLostLeader = metricsRegistry.registerMeter("nimbus:num-lost-leadership");
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/pacemaker/Pacemaker.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/pacemaker/Pacemaker.java b/storm-server/src/main/java/org/apache/storm/pacemaker/Pacemaker.java
index 2eea634..5144151 100644
--- a/storm-server/src/main/java/org/apache/storm/pacemaker/Pacemaker.java
+++ b/storm-server/src/main/java/org/apache/storm/pacemaker/Pacemaker.java
@@ -12,6 +12,7 @@
package org.apache.storm.pacemaker;
+import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import java.util.ArrayList;
@@ -27,35 +28,41 @@ import org.apache.storm.generated.HBServerMessageType;
import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.shade.uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
import org.apache.storm.utils.VersionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
public class Pacemaker implements IServerMessageHandler {
private static final Logger LOG = LoggerFactory.getLogger(Pacemaker.class);
- private final static Meter meterSendPulseCount = StormMetricsRegistry.registerMeter("pacemaker:send-pulse-count");
- private final static Meter meterTotalReceivedSize = StormMetricsRegistry.registerMeter("pacemaker:total-receive-size");
- private final static Meter meterGetPulseCount = StormMetricsRegistry.registerMeter("pacemaker:get-pulse=count");
- private final static Meter meterTotalSentSize = StormMetricsRegistry.registerMeter("pacemaker:total-sent-size");
- private final static Histogram histogramHeartbeatSize = StormMetricsRegistry.registerHistogram("pacemaker:heartbeat-size");
+ private final Meter meterSendPulseCount;
+ private final Meter meterTotalReceivedSize;
+ private final Meter meterGetPulseCount;
+ private final Meter meterTotalSentSize;
+ private final Histogram histogramHeartbeatSize;
private final Map<String, byte[]> heartbeats;
private final Map<String, Object> conf;
-
- public Pacemaker(Map<String, Object> conf) {
+ public Pacemaker(Map<String, Object> conf, StormMetricsRegistry metricsRegistry) {
heartbeats = new ConcurrentHashMap<>();
this.conf = conf;
- StormMetricsRegistry.registerGauge("pacemaker:size-total-keys", heartbeats::size);
- StormMetricsRegistry.startMetricsReporters(conf);
+ this.meterSendPulseCount = metricsRegistry.registerMeter("pacemaker:send-pulse-count");
+ this.meterTotalReceivedSize = metricsRegistry.registerMeter("pacemaker:total-receive-size");
+ this.meterGetPulseCount = metricsRegistry.registerMeter("pacemaker:get-pulse=count");
+ this.meterTotalSentSize = metricsRegistry.registerMeter("pacemaker:total-sent-size");
+ this.histogramHeartbeatSize = metricsRegistry.registerHistogram("pacemaker:heartbeat-size", new ExponentiallyDecayingReservoir());
+ metricsRegistry.registerGauge("pacemaker:size-total-keys", heartbeats::size);
}
public static void main(String[] args) {
SysOutOverSLF4J.sendSystemOutAndErrToSLF4J();
Map<String, Object> conf = ConfigUtils.overrideLoginConfigWithSystemProperty(ConfigUtils.readStormConfig());
- final Pacemaker serverHandler = new Pacemaker(conf);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ final Pacemaker serverHandler = new Pacemaker(conf, metricsRegistry);
serverHandler.launchServer();
+ metricsRegistry.startMetricsReporters(conf);
+ Utils.addShutdownHookWithForceKillIn1Sec(metricsRegistry::stopMetricsReporters);
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/Cluster.java b/storm-server/src/main/java/org/apache/storm/scheduler/Cluster.java
index e8d771d..b8bc69a 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/Cluster.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/Cluster.java
@@ -39,6 +39,7 @@ import org.apache.storm.networktopography.DefaultRackDNSToSwitchMapping;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceOffer;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
import org.apache.storm.scheduler.resource.normalization.NormalizedResources;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.ObjectReader;
@@ -80,17 +81,19 @@ public class Cluster implements ISchedulingState {
private final Map<String, Map<WorkerSlot, NormalizedResourceRequest>> nodeToScheduledResourcesCache;
private final Map<String, Set<WorkerSlot>> nodeToUsedSlotsCache;
private final Map<String, NormalizedResourceRequest> totalResourcesPerNodeCache = new HashMap<>();
+ private final ResourceMetrics resourceMetrics;
private SchedulerAssignmentImpl assignment;
private Set<String> blackListedHosts = new HashSet<>();
private INimbus inimbus;
public Cluster(
INimbus nimbus,
+ ResourceMetrics resourceMetrics,
Map<String, SupervisorDetails> supervisors,
Map<String, ? extends SchedulerAssignment> map,
Topologies topologies,
Map<String, Object> conf) {
- this(nimbus, supervisors, map, topologies, conf, null, null, null);
+ this(nimbus, resourceMetrics, supervisors, map, topologies, conf, null, null, null);
}
/**
@@ -99,6 +102,7 @@ public class Cluster implements ISchedulingState {
public Cluster(Cluster src) {
this(
src.inimbus,
+ src.resourceMetrics,
src.supervisors,
src.assignments,
src.topologies,
@@ -118,6 +122,7 @@ public class Cluster implements ISchedulingState {
public Cluster(Cluster src, Topologies topologies) {
this(
src.inimbus,
+ src.resourceMetrics,
src.supervisors,
src.assignments,
topologies,
@@ -129,6 +134,7 @@ public class Cluster implements ISchedulingState {
private Cluster(
INimbus nimbus,
+ ResourceMetrics resourceMetrics,
Map<String, SupervisorDetails> supervisors,
Map<String, ? extends SchedulerAssignment> assignments,
Topologies topologies,
@@ -137,6 +143,7 @@ public class Cluster implements ISchedulingState {
Set<String> blackListedHosts,
Map<String, List<String>> networkTopography) {
this.inimbus = nimbus;
+ this.resourceMetrics = resourceMetrics;
this.supervisors.putAll(supervisors);
this.nodeToScheduledResourcesCache = new HashMap<>(this.supervisors.size());
this.nodeToUsedSlotsCache = new HashMap<>(this.supervisors.size());
@@ -466,7 +473,7 @@ public class Cluster implements ISchedulingState {
for (SchedulerAssignment assignment: assignments.values()) {
for (Entry<WorkerSlot, WorkerResources> entry: assignment.getScheduledResources().entrySet()) {
if (sd.getId().equals(entry.getKey().getNodeId())) {
- ret.remove(entry.getValue());
+ ret.remove(entry.getValue(), getResourceMetrics());
}
}
}
@@ -812,7 +819,7 @@ public class Cluster implements ISchedulingState {
for (SupervisorDetails sup : supervisors.values()) {
if (!isBlackListed(sup.getId()) && !blacklistedSupervisorIds.contains(sup.getId())) {
available.add(sup.getTotalResources());
- available.remove(getAllScheduledResourcesForNode(sup.getId()));
+ available.remove(getAllScheduledResourcesForNode(sup.getId()), getResourceMetrics());
}
}
return available;
@@ -971,6 +978,10 @@ public class Cluster implements ISchedulingState {
nodeToUsedSlotsCache.computeIfAbsent(nodeId, MAKE_SET).add(workerSlot);
}
+ public ResourceMetrics getResourceMetrics() {
+ return resourceMetrics;
+ }
+
@Override
public NormalizedResourceRequest getAllScheduledResourcesForNode(String nodeId) {
return totalResourcesPerNodeCache.computeIfAbsent(nodeId, (nid) -> {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java b/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java
index 4ac1057..2700871 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/SupervisorDetails.java
@@ -17,6 +17,7 @@ import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceOffer;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/TopologyDetails.java b/storm-server/src/main/java/org/apache/storm/scheduler/TopologyDetails.java
index 5d781aa..757f256 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/TopologyDetails.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/TopologyDetails.java
@@ -26,8 +26,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.storm.Config;
-import org.apache.storm.Constants;
-import org.apache.storm.daemon.Acker;
import org.apache.storm.generated.Bolt;
import org.apache.storm.generated.ComponentCommon;
import org.apache.storm.generated.ComponentType;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java b/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java
index 5034f42..1e5b04d 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/BlacklistScheduler.java
@@ -42,6 +42,7 @@ public class BlacklistScheduler implements IScheduler {
public static final int DEFAULT_BLACKLIST_SCHEDULER_TOLERANCE_TIME = 300;
private static final Logger LOG = LoggerFactory.getLogger(BlacklistScheduler.class);
private final IScheduler underlyingScheduler;
+ private final StormMetricsRegistry metricsRegistry;
protected int toleranceTime;
protected int toleranceCount;
protected int resumeTime;
@@ -55,8 +56,9 @@ public class BlacklistScheduler implements IScheduler {
protected Set<String> blacklistHost;
private Map<String, Object> conf;
- public BlacklistScheduler(IScheduler underlyingScheduler) {
+ public BlacklistScheduler(IScheduler underlyingScheduler, StormMetricsRegistry metricsRegistry) {
this.underlyingScheduler = underlyingScheduler;
+ this.metricsRegistry = metricsRegistry;
}
@Override
@@ -89,7 +91,7 @@ public class BlacklistScheduler implements IScheduler {
blacklistHost = new HashSet<>();
//nimbus:num-blacklisted-supervisor + non-blacklisted supervisor = nimbus:num-supervisors
- StormMetricsRegistry.registerGauge("nimbus:num-blacklisted-supervisor", () -> blacklistHost.size());
+ metricsRegistry.registerGauge("nimbus:num-blacklisted-supervisor", () -> blacklistHost.size());
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/strategies/RasBlacklistStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/strategies/RasBlacklistStrategy.java b/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/strategies/RasBlacklistStrategy.java
index f7abc04..6c823ab 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/strategies/RasBlacklistStrategy.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/blacklist/strategies/RasBlacklistStrategy.java
@@ -71,7 +71,7 @@ public class RasBlacklistStrategy extends DefaultBlacklistStrategy {
//Now we need to free up some resources...
Map<String, SupervisorDetails> availableSupervisors = cluster.getSupervisors();
NormalizedResourceOffer shortage = new NormalizedResourceOffer(needed);
- shortage.remove(available);
+ shortage.remove(available, cluster.getResourceMetrics());
int shortageSlots = neededSlots - availableSlots;
LOG.debug("Need {} and {} slots.", needed, neededSlots);
LOG.debug("Available {} and {} slots.", available, availableSlots);
@@ -86,7 +86,7 @@ public class RasBlacklistStrategy extends DefaultBlacklistStrategy {
NormalizedResourcesWithMemory sdAvailable = cluster.getAvailableResources(sd);
int sdAvailableSlots = cluster.getAvailablePorts(sd).size();
readyToRemove.add(supervisor);
- shortage.remove(sdAvailable);
+ shortage.remove(sdAvailable, cluster.getResourceMetrics());
shortageSlots -= sdAvailableSlots;
LOG.debug("Releasing {} with {} and {} slots leaving {} and {} slots to go", supervisor,
sdAvailable, sdAvailableSlots, shortage, shortageSlots);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/RAS_Node.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/RAS_Node.java
index 5975780..89c0460 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/RAS_Node.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/RAS_Node.java
@@ -445,7 +445,7 @@ public class RAS_Node {
public NormalizedResourceOffer getTotalAvailableResources() {
if (sup != null) {
NormalizedResourceOffer availableResources = new NormalizedResourceOffer(sup.getTotalResources());
- if (availableResources.remove(cluster.getAllScheduledResourcesForNode(sup.getId()))) {
+ if (availableResources.remove(cluster.getAllScheduledResourcesForNode(sup.getId()), cluster.getResourceMetrics())) {
if (!loggedUnderageUsage) {
LOG.error("Resources on {} became negative and was clamped to 0 {}.", hostname, availableResources);
loggedUnderageUsage = true;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java
index 7ef5f74..366995e 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/ResourceUtils.java
@@ -39,8 +39,7 @@ public class ResourceUtils {
return null;
}
- public static Map<String, NormalizedResourceRequest> getBoltsResources(StormTopology topology,
- Map<String, Object> topologyConf) {
+ public static Map<String, NormalizedResourceRequest> getBoltsResources(StormTopology topology, Map<String, Object> topologyConf) {
Map<String, NormalizedResourceRequest> boltResources = new HashMap<>();
if (topology.get_bolts() != null) {
for (Map.Entry<String, Bolt> bolt : topology.get_bolts().entrySet()) {
@@ -55,8 +54,8 @@ public class ResourceUtils {
return boltResources;
}
- public static NormalizedResourceRequest getSpoutResources(StormTopology topology, Map<String, Object> topologyConf,
- String componentId) {
+ public static NormalizedResourceRequest getSpoutResources(StormTopology topology,
+ Map<String, Object> topologyConf, String componentId) {
if (topology.get_spouts() != null) {
SpoutSpec spout = topology.get_spouts().get(componentId);
return new NormalizedResourceRequest(spout.get_common(), topologyConf, componentId);
@@ -65,7 +64,7 @@ public class ResourceUtils {
}
public static Map<String, NormalizedResourceRequest> getSpoutsResources(StormTopology topology,
- Map<String, Object> topologyConf) {
+ Map<String, Object> topologyConf) {
Map<String, NormalizedResourceRequest> spoutResources = new HashMap<>();
if (topology.get_spouts() != null) {
for (Map.Entry<String, SpoutSpec> spout : topology.get_spouts().entrySet()) {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
index c680be7..707c893 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOffer.java
@@ -84,14 +84,15 @@ public class NormalizedResourceOffer implements NormalizedResourcesWithMemory {
/**
* Remove the resources in other from this.
* @param other the resources to be removed.
+ * @param resourceMetrics The resource related metrics
* @return true if one or more resources in other were larger than available resources in this, else false.
*/
- public boolean remove(NormalizedResourcesWithMemory other) {
- boolean negativeResources = normalizedResources.remove(other.getNormalizedResources());
+ public boolean remove(NormalizedResourcesWithMemory other, ResourceMetrics resourceMetrics) {
+ boolean negativeResources = normalizedResources.remove(other.getNormalizedResources(), resourceMetrics);
totalMemoryMb -= other.getTotalMemoryMb();
if (totalMemoryMb < 0.0) {
negativeResources = true;
- NormalizedResources.numNegativeResourceEvents.mark();
+ resourceMetrics.getNegativeResourceEventsMeter().mark();
totalMemoryMb = 0.0;
}
return negativeResources;
@@ -100,14 +101,15 @@ public class NormalizedResourceOffer implements NormalizedResourcesWithMemory {
/**
* Remove the resources in other from this.
* @param other the resources to be removed.
+ * @param resourceMetrics The resource related metrics
* @return true if one or more resources in other were larger than available resources in this, else false.
*/
- public boolean remove(WorkerResources other) {
+ public boolean remove(WorkerResources other, ResourceMetrics resourceMetrics) {
boolean negativeResources = normalizedResources.remove(other);
totalMemoryMb -= (other.get_mem_off_heap() + other.get_mem_on_heap());
if (totalMemoryMb < 0.0) {
negativeResources = true;
- NormalizedResources.numNegativeResourceEvents.mark();
+ resourceMetrics.getNegativeResourceEventsMeter().mark();
totalMemoryMb = 0.0;
}
return negativeResources;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceRequest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceRequest.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceRequest.java
index 14c6846..478a8be 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceRequest.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceRequest.java
@@ -43,7 +43,7 @@ public class NormalizedResourceRequest implements NormalizedResourcesWithMemory
private double offHeap;
private NormalizedResourceRequest(Map<String, ? extends Number> resources,
- Map<String, Double> defaultResources) {
+ Map<String, Double> defaultResources) {
if (resources == null && defaultResources == null) {
onHeap = 0.0;
offHeap = 0.0;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
index 1e6af6f..6419406 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/NormalizedResources.java
@@ -35,16 +35,13 @@ import org.slf4j.LoggerFactory;
public class NormalizedResources {
private static final Logger LOG = LoggerFactory.getLogger(NormalizedResources.class);
- public static final Meter numNegativeResourceEvents = StormMetricsRegistry.registerMeter("nimbus:num-negative-resource-events");
-
-
public static ResourceNameNormalizer RESOURCE_NAME_NORMALIZER;
private static ResourceMapArrayBridge RESOURCE_MAP_ARRAY_BRIDGE;
static {
resetResourceNames();
}
-
+
private double cpu;
private double[] otherResources;
@@ -130,14 +127,15 @@ public class NormalizedResources {
* Remove the other resources from this. This is the same as subtracting the resources in other from this.
*
* @param other the resources we want removed.
+ * @param resourceMetrics The resource related metrics
* @return true if the resources would have gone negative, but were clamped to 0.
*/
- public boolean remove(NormalizedResources other) {
+ public boolean remove(NormalizedResources other, ResourceMetrics resourceMetrics) {
boolean ret = false;
this.cpu -= other.cpu;
if (cpu < 0.0) {
ret = true;
- numNegativeResourceEvents.mark();
+ resourceMetrics.getNegativeResourceEventsMeter().mark();
cpu = 0.0;
}
int otherLength = other.otherResources.length;
@@ -146,7 +144,7 @@ public class NormalizedResources {
otherResources[i] -= other.otherResources[i];
if (otherResources[i] < 0.0) {
ret = true;
- numNegativeResourceEvents.mark();
+ resourceMetrics.getNegativeResourceEventsMeter().mark();
otherResources[i] = 0.0;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/ResourceMetrics.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/ResourceMetrics.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/ResourceMetrics.java
new file mode 100644
index 0000000..851499b
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/normalization/ResourceMetrics.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2018 The Apache Software Foundation.
+ *
+ * Licensed 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.storm.scheduler.resource.normalization;
+
+import com.codahale.metrics.Meter;
+import org.apache.storm.metric.StormMetricsRegistry;
+
+public class ResourceMetrics {
+
+ private final Meter numNegativeResourceEvents;
+
+ public ResourceMetrics(StormMetricsRegistry metricsRegistry) {
+ numNegativeResourceEvents = metricsRegistry.registerMeter("nimbus:num-negative-resource-events");
+ }
+
+ public Meter getNegativeResourceEventsMeter() {
+ return numNegativeResourceEvents;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
index 59b0f31..b5da8db 100644
--- a/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
+++ b/storm-server/src/main/java/org/apache/storm/scheduler/resource/strategies/scheduling/BaseResourceAwareStrategy.java
@@ -45,6 +45,7 @@ import org.apache.storm.scheduler.resource.SchedulingResult;
import org.apache.storm.scheduler.resource.SchedulingStatus;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceOffer;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
import org.apache.storm.shade.com.google.common.collect.Sets;
import org.slf4j.Logger;
@@ -632,12 +633,14 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
*/
static class AllResources {
List<ObjectResources> objectResources = new LinkedList<>();
- NormalizedResourceOffer availableResourcesOverall = new NormalizedResourceOffer();
- NormalizedResourceOffer totalResourcesOverall = new NormalizedResourceOffer();
+ final NormalizedResourceOffer availableResourcesOverall;
+ final NormalizedResourceOffer totalResourcesOverall;
String identifier;
public AllResources(String identifier) {
this.identifier = identifier;
+ this.availableResourcesOverall = new NormalizedResourceOffer();
+ this.totalResourcesOverall = new NormalizedResourceOffer();
}
public AllResources(AllResources other) {
@@ -666,12 +669,14 @@ public abstract class BaseResourceAwareStrategy implements IStrategy {
*/
static class ObjectResources {
public final String id;
- public NormalizedResourceOffer availableResources = new NormalizedResourceOffer();
- public NormalizedResourceOffer totalResources = new NormalizedResourceOffer();
+ public NormalizedResourceOffer availableResources;
+ public NormalizedResourceOffer totalResources;
public double effectiveResources = 0.0;
public ObjectResources(String id) {
this.id = id;
+ this.availableResources = new NormalizedResourceOffer();
+ this.totalResources = new NormalizedResourceOffer();
}
public ObjectResources(ObjectResources other) {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java b/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java
index 03fec1f..6dc33e8 100644
--- a/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java
+++ b/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java
@@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.storm.blobstore.BlobStore;
import org.apache.storm.cluster.IStormClusterState;
import org.apache.storm.daemon.nimbus.TopoCache;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.nimbus.LeaderListenerCallback;
import org.apache.storm.nimbus.NimbusInfo;
@@ -45,10 +46,12 @@ public class LeaderElectorImp implements ILeaderElector {
private final TopoCache tc;
private final IStormClusterState clusterState;
private final List<ACL> acls;
+ private final StormMetricsRegistry metricsRegistry;
public LeaderElectorImp(Map<String, Object> conf, List<String> servers, CuratorFramework zk, String leaderlockPath, String id,
AtomicReference<LeaderLatch> leaderLatch, AtomicReference<LeaderLatchListener> leaderLatchListener,
- BlobStore blobStore, final TopoCache tc, IStormClusterState clusterState, List<ACL> acls) {
+ BlobStore blobStore, final TopoCache tc, IStormClusterState clusterState, List<ACL> acls,
+ StormMetricsRegistry metricsRegistry) {
this.conf = conf;
this.servers = servers;
this.zk = zk;
@@ -60,6 +63,7 @@ public class LeaderElectorImp implements ILeaderElector {
this.tc = tc;
this.clusterState = clusterState;
this.acls = acls;
+ this.metricsRegistry = metricsRegistry;
}
@Override
@@ -72,7 +76,8 @@ public class LeaderElectorImp implements ILeaderElector {
// if this latch is already closed, we need to create new instance.
if (LeaderLatch.State.CLOSED.equals(leaderLatch.get().getState())) {
leaderLatch.set(new LeaderLatch(zk, leaderlockPath));
- LeaderListenerCallback callback = new LeaderListenerCallback(conf, zk, leaderLatch.get(), blobStore, tc, clusterState, acls);
+ LeaderListenerCallback callback = new LeaderListenerCallback(conf, zk, leaderLatch.get(), blobStore, tc, clusterState, acls,
+ metricsRegistry);
leaderLatchListener.set(Zookeeper.leaderLatchListenerImpl(callback));
LOG.info("LeaderLatch was in closed state. Resetted the leaderLatch and listeners.");
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java b/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java
index f30c246..5468573 100644
--- a/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-server/src/main/java/org/apache/storm/zookeeper/Zookeeper.java
@@ -31,6 +31,7 @@ import org.apache.storm.Config;
import org.apache.storm.blobstore.BlobStore;
import org.apache.storm.cluster.IStormClusterState;
import org.apache.storm.daemon.nimbus.TopoCache;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.nimbus.LeaderListenerCallback;
import org.apache.storm.nimbus.NimbusInfo;
@@ -146,13 +147,14 @@ public class Zookeeper {
* @throws UnknownHostException
*/
public static ILeaderElector zkLeaderElector(Map<String, Object> conf, CuratorFramework zkClient, BlobStore blobStore,
- final TopoCache tc, IStormClusterState clusterState, List<ACL> acls) throws
- UnknownHostException {
- return _instance.zkLeaderElectorImpl(conf, zkClient, blobStore, tc, clusterState, acls);
+ final TopoCache tc, IStormClusterState clusterState, List<ACL> acls,
+ StormMetricsRegistry metricsRegistry) throws UnknownHostException {
+ return _instance.zkLeaderElectorImpl(conf, zkClient, blobStore, tc, clusterState, acls, metricsRegistry);
}
protected ILeaderElector zkLeaderElectorImpl(Map<String, Object> conf, CuratorFramework zk, BlobStore blobStore,
- final TopoCache tc, IStormClusterState clusterState, List<ACL> acls) throws
+ final TopoCache tc, IStormClusterState clusterState, List<ACL> acls,
+ StormMetricsRegistry metricsRegistry) throws
UnknownHostException {
List<String> servers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
String leaderLockPath = "/leader-lock";
@@ -160,9 +162,9 @@ public class Zookeeper {
AtomicReference<LeaderLatch> leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id));
AtomicReference<LeaderLatchListener> leaderLatchListenerAtomicReference =
new AtomicReference<>(leaderLatchListenerImpl(
- new LeaderListenerCallback(conf, zk, leaderLatchAtomicReference.get(), blobStore, tc, clusterState, acls)));
+ new LeaderListenerCallback(conf, zk, leaderLatchAtomicReference.get(), blobStore, tc, clusterState, acls, metricsRegistry)));
return new LeaderElectorImp(conf, servers, zk, leaderLockPath, id, leaderLatchAtomicReference,
- leaderLatchListenerAtomicReference, blobStore, tc, clusterState, acls);
+ leaderLatchListenerAtomicReference, blobStore, tc, clusterState, acls, metricsRegistry);
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/PacemakerTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/PacemakerTest.java b/storm-server/src/test/java/org/apache/storm/PacemakerTest.java
index 5f7f8e9..dfd53be 100644
--- a/storm-server/src/test/java/org/apache/storm/PacemakerTest.java
+++ b/storm-server/src/test/java/org/apache/storm/PacemakerTest.java
@@ -20,6 +20,7 @@ import org.apache.storm.generated.HBMessage;
import org.apache.storm.generated.HBMessageData;
import org.apache.storm.generated.HBPulse;
import org.apache.storm.generated.HBServerMessageType;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.pacemaker.Pacemaker;
import org.apache.storm.utils.Utils;
import org.junit.Assert;
@@ -31,15 +32,16 @@ public class PacemakerTest {
private HBMessage hbMessage;
private int mid;
private Random random;
+ private Pacemaker handler;
@Before
public void init() {
random = new Random(100);
+ handler = new Pacemaker(new ConcurrentHashMap<>(), new StormMetricsRegistry());
}
@Test
public void testServerCreatePath() {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
messageWithRandId(HBServerMessageType.CREATE_PATH, HBMessageData.path("/testpath"));
HBMessage response = handler.handleMessage(hbMessage, true);
Assert.assertEquals(mid, response.get_message_id());
@@ -49,7 +51,6 @@ public class PacemakerTest {
@Test
public void testServerExistsFalse() {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
messageWithRandId(HBServerMessageType.EXISTS, HBMessageData.path("/testpath"));
HBMessage badResponse = handler.handleMessage(hbMessage, false);
HBMessage goodResponse = handler.handleMessage(hbMessage, true);
@@ -65,7 +66,6 @@ public class PacemakerTest {
public void testServerExistsTrue() {
String path = "/exists_path";
String dataString = "pulse data";
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
HBPulse hbPulse = new HBPulse();
hbPulse.set_id(path);
hbPulse.set_details(Utils.javaSerialize(dataString));
@@ -87,7 +87,6 @@ public class PacemakerTest {
public void testServerSendPulseGetPulse() throws UnsupportedEncodingException {
String path = "/pulsepath";
String dataString = "pulse data";
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
HBPulse hbPulse = new HBPulse();
hbPulse.set_id(path);
hbPulse.set_details(dataString.getBytes("UTF-8"));
@@ -106,7 +105,6 @@ public class PacemakerTest {
@Test
public void testServerGetAllPulseForPath() {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
messageWithRandId(HBServerMessageType.GET_ALL_PULSE_FOR_PATH, HBMessageData.path("/testpath"));
HBMessage badResponse = handler.handleMessage(hbMessage, false);
HBMessage goodResponse = handler.handleMessage(hbMessage, true);
@@ -120,7 +118,6 @@ public class PacemakerTest {
@Test
public void testServerGetAllNodesForPath() throws UnsupportedEncodingException {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
makeNode(handler, "/some-root-path/foo");
makeNode(handler, "/some-root-path/bar");
makeNode(handler, "/some-root-path/baz");
@@ -162,7 +159,6 @@ public class PacemakerTest {
@Test
public void testServerGetPulse() throws UnsupportedEncodingException {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
makeNode(handler, "/some-root/GET_PULSE");
messageWithRandId(HBServerMessageType.GET_PULSE, HBMessageData.path("/some-root/GET_PULSE"));
HBMessage badResponse = handler.handleMessage(hbMessage, false);
@@ -180,7 +176,6 @@ public class PacemakerTest {
@Test
public void testServerDeletePath() throws UnsupportedEncodingException {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
makeNode(handler, "/some-root/DELETE_PATH/foo");
makeNode(handler, "/some-root/DELETE_PATH/bar");
makeNode(handler, "/some-root/DELETE_PATH/baz");
@@ -202,7 +197,6 @@ public class PacemakerTest {
@Test
public void testServerDeletePulseId() throws UnsupportedEncodingException {
- Pacemaker handler = new Pacemaker(new ConcurrentHashMap());
makeNode(handler, "/some-root/DELETE_PULSE_ID/foo");
makeNode(handler, "/some-root/DELETE_PULSE_ID/bar");
makeNode(handler, "/some-root/DELETE_PULSE_ID/baz");
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/daemon/drpc/DRPCTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/drpc/DRPCTest.java b/storm-server/src/test/java/org/apache/storm/daemon/drpc/DRPCTest.java
index e4b586b..a0825d9 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/drpc/DRPCTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/drpc/DRPCTest.java
@@ -47,6 +47,8 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+import org.apache.storm.metric.StormMetricsRegistry;
+
public class DRPCTest {
private static final ExecutorService exec = Executors.newCachedThreadPool();
@@ -80,7 +82,7 @@ public class DRPCTest {
@Test
public void testGoodBlocking() throws Exception {
- try (DRPC server = new DRPC(null, 100)) {
+ try (DRPC server = new DRPC(new StormMetricsRegistry(), null, 100)) {
Future<String> found = exec.submit(() -> server.executeBlocking("testing", "test"));
DRPCRequest request = getNextAvailableRequest(server, "testing");
assertNotNull(request);
@@ -94,7 +96,7 @@ public class DRPCTest {
@Test
public void testFailedBlocking() throws Exception {
- try (DRPC server = new DRPC(null, 100)) {
+ try (DRPC server = new DRPC(new StormMetricsRegistry(), null, 100)) {
Future<String> found = exec.submit(() -> server.executeBlocking("testing", "test"));
DRPCRequest request = getNextAvailableRequest(server, "testing");
assertNotNull(request);
@@ -116,7 +118,7 @@ public class DRPCTest {
@Test
public void testDequeueAfterTimeout() throws Exception {
long timeout = 1000;
- try (DRPC server = new DRPC(null, timeout)) {
+ try (DRPC server = new DRPC(new StormMetricsRegistry(), null, timeout)) {
long start = Time.currentTimeMillis();
try {
server.executeBlocking("testing", "test");
@@ -136,7 +138,7 @@ public class DRPCTest {
@Test
public void testDeny() throws Exception {
- try (DRPC server = new DRPC(new DenyAuthorizer(), 100)) {
+ try (DRPC server = new DRPC(new StormMetricsRegistry(), new DenyAuthorizer(), 100)) {
assertThrows(() -> server.executeBlocking("testing", "test"), AuthorizationException.class);
assertThrows(() -> server.fetchRequest("testing"), AuthorizationException.class);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java
index 1094e1c..1c313be 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/BasicContainerTest.java
@@ -41,6 +41,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.ConfigUtils;
public class BasicContainerTest {
@@ -102,8 +103,8 @@ public class BasicContainerTest {
LocalState ls = mock(LocalState.class);
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, null, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
//null worker id means generate one...
assertNotNull(mc._workerId);
@@ -133,8 +134,8 @@ public class BasicContainerTest {
when(ops.doRequiredTopoFilesExist(superConf, topoId)).thenReturn(true);
MockBasicContainer mc = new MockBasicContainer(ContainerType.RECOVER_FULL, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, null, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
assertEquals(workerId, mc._workerId);
}
@@ -155,7 +156,8 @@ public class BasicContainerTest {
try {
new MockBasicContainer(ContainerType.RECOVER_FULL, new HashMap<String, Object>(),
- "SUPERVISOR", supervisorPort, port, la, null, ls, null, new HashMap<>(), null, "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, null, new StormMetricsRegistry(),
+ new HashMap<>(), null, "profile");
fail("Container recovered worker incorrectly");
} catch (ContainerRecoveryException e) {
//Expected
@@ -182,7 +184,7 @@ public class BasicContainerTest {
when(ls.getApprovedWorkers()).thenReturn(new HashMap<>(workerState));
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops,
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(), new HashMap<>(), ops,
"profile");
mc.cleanUp();
@@ -218,8 +220,8 @@ public class BasicContainerTest {
LocalState ls = mock(LocalState.class);
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
//HEAP DUMP
ProfileRequest req = new ProfileRequest();
@@ -329,9 +331,8 @@ public class BasicContainerTest {
checkpoint(() -> {
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new
- HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
mc.launch();
@@ -432,9 +433,8 @@ public class BasicContainerTest {
checkpoint(() -> {
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new
- HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
mc.launch();
@@ -534,9 +534,8 @@ public class BasicContainerTest {
checkpoint(() -> {
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new
- HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
mc.launch();
@@ -612,8 +611,8 @@ public class BasicContainerTest {
LocalState ls = mock(LocalState.class);
MockBasicContainer mc = new MockBasicContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new HashMap<>(), ops,
- "profile");
+ "SUPERVISOR", supervisorPort, port, la, null, ls, workerId, new StormMetricsRegistry(),
+ new HashMap<>(), ops, "profile");
assertListEquals(Arrays.asList(
"-Xloggc:/tmp/storm/logs/gc.worker-9999-s-01-w-01-9999.log",
@@ -656,10 +655,10 @@ public class BasicContainerTest {
public final List<CommandRun> workerCmds = new ArrayList<>();
public MockBasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort,
int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- LocalState localState, String workerId, Map<String, Object> topoConf, AdvancedFSOps ops,
- String profileCmd) throws IOException {
+ LocalState localState, String workerId, StormMetricsRegistry metricsRegistry,
+ Map<String, Object> topoConf, AdvancedFSOps ops, String profileCmd) throws IOException {
super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState,
- workerId, topoConf, ops, profileCmd);
+ workerId, metricsRegistry,new ContainerMemoryTracker(metricsRegistry), topoConf, ops, profileCmd);
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
index b510838..cf5ec8d 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/ContainerTest.java
@@ -42,6 +42,8 @@ import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
+import org.apache.storm.metric.StormMetricsRegistry;
+
public class ContainerTest {
private static final Joiner PATH_JOIN = Joiner.on(File.separator).skipNulls();
private static final String DOUBLE_SEP = File.separator + File.separator;
@@ -71,7 +73,7 @@ public class ContainerTest {
LocalAssignment la = new LocalAssignment();
la.set_topology_id(topoId);
MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", 6628, 8080, la, null, "worker", new HashMap<>(), ops);
+ "SUPERVISOR", 6628, 8080, la, null, "worker", new HashMap<>(), ops, new StormMetricsRegistry());
mc.kill();
assertEquals(Collections.EMPTY_LIST, mc.killedPids);
assertEquals(Collections.EMPTY_LIST, mc.forceKilledPids);
@@ -134,7 +136,7 @@ public class ContainerTest {
la.set_topology_id(topoId);
la.set_owner(user);
MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", 6628, 8080, la, null, workerId, topoConf, ops);
+ "SUPERVISOR", 6628, 8080, la, null, workerId, topoConf, ops, new StormMetricsRegistry());
mc.setup();
@@ -205,7 +207,7 @@ public class ContainerTest {
la.set_owner(user);
la.set_topology_id(topoId);
MockContainer mc = new MockContainer(ContainerType.LAUNCH, superConf,
- "SUPERVISOR", supervisorPort, port, la, iso, workerId, topoConf, ops);
+ "SUPERVISOR", supervisorPort, port, la, iso, workerId, topoConf, ops, new StormMetricsRegistry());
mc.allPids.add(pid);
mc.cleanUp();
@@ -226,9 +228,9 @@ public class ContainerTest {
public final Set<Long> allPids = new HashSet<>();
protected MockContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort,
int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- String workerId, Map<String, Object> topoConf, AdvancedFSOps ops) throws IOException {
+ String workerId, Map<String, Object> topoConf, AdvancedFSOps ops, StormMetricsRegistry metricsRegistry) throws IOException {
super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, workerId,
- topoConf, ops);
+ topoConf, ops, metricsRegistry, new ContainerMemoryTracker(new StormMetricsRegistry()));
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java
index 6f09f3f..21a9bc2 100644
--- a/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java
+++ b/storm-server/src/test/java/org/apache/storm/daemon/supervisor/SlotTest.java
@@ -52,6 +52,8 @@ import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+
public class SlotTest {
private static final Logger LOG = LoggerFactory.getLogger(SlotTest.class);
@@ -147,9 +149,11 @@ public class SlotTest {
BlobChangingCallback cb = mock(BlobChangingCallback.class);
ContainerLauncher containerLauncher = mock(ContainerLauncher.class);
ISupervisor iSuper = mock(ISupervisor.class);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
StaticState staticState = new StaticState(localizer, 1000, 1000, 1000, 1000,
- containerLauncher, "localhost", 8080, iSuper, state, cb, null, null);
- DynamicState dynamicState = new DynamicState(null, null, null);
+ containerLauncher, "localhost", 8080, iSuper, state, cb, null, null,
+ slotMetrics);
+ DynamicState dynamicState = new DynamicState(null, null, null, slotMetrics);
DynamicState nextState = Slot.handleEmpty(dynamicState, staticState);
assertEquals(MachineState.EMPTY, nextState.state);
assertTrue(Time.currentTimeMillis() > 1000);
@@ -179,9 +183,10 @@ public class SlotTest {
when(localizer.requestDownloadTopologyBlobs(newAssignment, port, cb)).thenReturn(blobFuture);
ISupervisor iSuper = mock(ISupervisor.class);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
- DynamicState dynamicState = new DynamicState(null, null, null)
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, slotMetrics);
+ DynamicState dynamicState = new DynamicState(null, null, null, slotMetrics)
.withNewAssignment(newAssignment);
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
@@ -248,9 +253,10 @@ public class SlotTest {
ISupervisor iSuper = mock(ISupervisor.class);
LocalState state = mock(LocalState.class);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
- DynamicState dynamicState = new DynamicState(assignment, container, assignment);
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, slotMetrics);
+ DynamicState dynamicState = new DynamicState(assignment, container, assignment, slotMetrics);
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
assertEquals(MachineState.KILL_AND_RELAUNCH, nextState.state);
@@ -309,9 +315,10 @@ public class SlotTest {
when(localizer.requestDownloadTopologyBlobs(nAssignment, port, cb)).thenReturn(blobFuture);
ISupervisor iSuper = mock(ISupervisor.class);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
- DynamicState dynamicState = new DynamicState(cAssignment, cContainer, nAssignment);
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, slotMetrics);
+ DynamicState dynamicState = new DynamicState(cAssignment, cContainer, nAssignment, slotMetrics);
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
assertEquals(MachineState.KILL, nextState.state);
@@ -390,9 +397,10 @@ public class SlotTest {
ISupervisor iSuper = mock(ISupervisor.class);
LocalState state = mock(LocalState.class);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
- DynamicState dynamicState = new DynamicState(cAssignment, cContainer, null);
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, slotMetrics);
+ DynamicState dynamicState = new DynamicState(cAssignment, cContainer, null, slotMetrics);
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
assertEquals(MachineState.KILL, nextState.state);
@@ -452,7 +460,7 @@ public class SlotTest {
ISupervisor iSuper = mock(ISupervisor.class);
LocalState state = mock(LocalState.class);
StaticState staticState = new StaticState(localizer, 5000, 120000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, new SlotMetrics(new StormMetricsRegistry()));
Set<TopoProfileAction> profileActions = new HashSet<>();
ProfileRequest request = new ProfileRequest();
request.set_action(ProfileAction.JPROFILE_STOP);
@@ -467,8 +475,8 @@ public class SlotTest {
Set<TopoProfileAction> expectedPending = new HashSet<>();
expectedPending.add(profile);
-
- DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment)
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
+ DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment, slotMetrics)
.withProfileActions(profileActions, Collections.<TopoProfileAction>emptySet());
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
@@ -534,7 +542,7 @@ public class SlotTest {
ISupervisor iSuper = mock(ISupervisor.class);
long heartbeatTimeoutMs = 5000;
StaticState staticState = new StaticState(localizer, heartbeatTimeoutMs, 120_000, 1000, 1000,
- containerLauncher, "localhost", port, iSuper, state, cb, null, null);
+ containerLauncher, "localhost", port, iSuper, state, cb, null, null, new SlotMetrics(new StormMetricsRegistry()));
Set<Slot.BlobChanging> changing = new HashSet<>();
@@ -549,7 +557,8 @@ public class SlotTest {
GoodToGo.GoodToGoLatch otherJarLatch = mock(GoodToGo.GoodToGoLatch.class);
changing.add(new Slot.BlobChanging(otherAssignment, otherJar, otherJarLatch));
- DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment).withChangingBlobs(changing);
+ SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry());
+ DynamicState dynamicState = new DynamicState(cAssignment, cContainer, cAssignment, slotMetrics).withChangingBlobs(changing);
DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState);
assertEquals(MachineState.KILL_BLOB_UPDATE, nextState.state);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java
index 8654d25..243499a 100644
--- a/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java
+++ b/storm-server/src/test/java/org/apache/storm/localizer/AsyncLocalizerTest.java
@@ -79,6 +79,8 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
+import org.apache.storm.metric.StormMetricsRegistry;
+
public class AsyncLocalizerTest {
private static final Logger LOG = LoggerFactory.getLogger(AsyncLocalizerTest.class);
private final String user1 = "user1";
@@ -117,7 +119,7 @@ public class AsyncLocalizerTest {
ReflectionUtils mockedRU = mock(ReflectionUtils.class);
ServerUtils mockedU = mock(ServerUtils.class);
- AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, getTestLocalizerRoot()));
+ AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, getTestLocalizerRoot(), new StormMetricsRegistry()));
LocallyCachedTopologyBlob jarBlob = mock(LocallyCachedTopologyBlob.class);
doReturn(jarBlob).when(bl).getTopoJar(topoId, la.get_owner());
when(jarBlob.getLocalVersion()).thenReturn(-1L);
@@ -213,10 +215,11 @@ public class AsyncLocalizerTest {
topoConf.put(Config.TOPOLOGY_NAME, topoName);
List<LocalizedResource> localizedList = new ArrayList<>();
- LocalizedResource simpleLocal = new LocalizedResource(simpleKey, Paths.get(localizerRoot), false, ops, conf, user);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ LocalizedResource simpleLocal = new LocalizedResource(simpleKey, Paths.get(localizerRoot), false, ops, conf, user, metricsRegistry);
localizedList.add(simpleLocal);
- AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, localizerRoot));
+ AsyncLocalizer bl = spy(new AsyncLocalizer(conf, ops, localizerRoot, metricsRegistry));
ConfigUtils orig = ConfigUtils.setInstance(mockedCU);
try {
when(mockedCU.supervisorStormDistRootImpl(conf, topoId)).thenReturn(stormRoot);
@@ -889,7 +892,7 @@ public class AsyncLocalizerTest {
class TestLocalizer extends AsyncLocalizer {
TestLocalizer(Map<String, Object> conf, String baseDir) throws IOException {
- super(conf, AdvancedFSOps.make(conf), baseDir);
+ super(conf, AdvancedFSOps.make(conf), baseDir, new StormMetricsRegistry());
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java
index f41c3df..db67e04 100644
--- a/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java
+++ b/storm-server/src/test/java/org/apache/storm/localizer/LocalizedResourceRetentionSetTest.java
@@ -32,6 +32,8 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
+import org.apache.storm.metric.StormMetricsRegistry;
+
public class LocalizedResourceRetentionSetTest {
@Test
@@ -43,9 +45,10 @@ public class LocalizedResourceRetentionSetTest {
IAdvancedFSOps ops = mock(IAdvancedFSOps.class);
LocalizedResourceRetentionSet lrretset = new LocalizedResourceRetentionSet(10);
ConcurrentMap<String, LocalizedResource> lrset = new ConcurrentHashMap<>();
- LocalizedResource localresource1 = new LocalizedResource("key1", Paths.get("testfile1"), false, ops, conf, user);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ LocalizedResource localresource1 = new LocalizedResource("key1", Paths.get("testfile1"), false, ops, conf, user, metricsRegistry);
localresource1.addReference(pna1, null);
- LocalizedResource localresource2 = new LocalizedResource("key2", Paths.get("testfile2"), false, ops, conf, user);
+ LocalizedResource localresource2 = new LocalizedResource("key2", Paths.get("testfile2"), false, ops, conf, user, metricsRegistry);
localresource2.addReference(pna1, null);
// check adding reference to local resource with topology of same name
localresource2.addReference(pna2, null);
@@ -81,17 +84,18 @@ public class LocalizedResourceRetentionSetTest {
LocalizedResourceRetentionSet lrretset = spy(new LocalizedResourceRetentionSet(10));
ConcurrentMap<String, LocalizedResource> lrFiles = new ConcurrentHashMap<>();
ConcurrentMap<String, LocalizedResource> lrArchives = new ConcurrentHashMap<>();
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
// no reference to key1
LocalizedResource localresource1 = new LocalizedResource("key1", Paths.get("./target/TESTING/testfile1"), false, ops, conf,
- user);
+ user, metricsRegistry);
localresource1.setSize(10);
// no reference to archive1
LocalizedResource archiveresource1 = new LocalizedResource("archive1", Paths.get("./target/TESTING/testarchive1"), true, ops,
- conf, user);
+ conf, user, metricsRegistry);
archiveresource1.setSize(20);
// reference to key2
LocalizedResource localresource2 = new LocalizedResource("key2", Paths.get("./target/TESTING/testfile2"), false, ops, conf,
- user);
+ user, metricsRegistry);
localresource2.addReference(pna1, null);
// check adding reference to local resource with topology of same name
localresource2.addReference(pna1, null);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/metric/StormMetricsRegistryTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metric/StormMetricsRegistryTest.java b/storm-server/src/test/java/org/apache/storm/metric/StormMetricsRegistryTest.java
deleted file mode 100644
index 5d9b3e4..0000000
--- a/storm-server/src/test/java/org/apache/storm/metric/StormMetricsRegistryTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.metric;
-
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.Metric;
-import com.codahale.metrics.MetricSet;
-import com.codahale.metrics.Timer;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static com.codahale.metrics.MetricRegistry.name;
-import static org.junit.jupiter.api.Assertions.*;
-
-class StormMetricsRegistryTest {
- private static final Logger LOG = LoggerFactory.getLogger(StormMetricsRegistryTest.class);
-
- private static final String OUTER_METER = "outerMeter";
- private static final String INNER_SET = "innerSet";
- private static final String OUTER_TIMER = "outerTimer";
- private static final String INNER_METER = "innerMeter";
- private static final String INNER_TIMER = "innerTimer";
- private static final MetricSet OUTER = newMetricSetInstance();
-
- @Test
- void registerMetricSet() {
- Meter existingInnerMeter = StormMetricsRegistry.registerMeter(name(INNER_SET, INNER_METER));
-
- LOG.info("register outer set");
- StormMetricsRegistry.registerMetricSet(OUTER);
- assertSame(OUTER.getMetrics().get(OUTER_TIMER), StormMetricsRegistry.REGISTRY.getMetrics().get(OUTER_TIMER));
- assertSame(OUTER.getMetrics().get(OUTER_METER), StormMetricsRegistry.REGISTRY.getMetrics().get(OUTER_METER));
- assertSame(((MetricSet) OUTER.getMetrics().get(INNER_SET)).getMetrics().get(INNER_TIMER),
- StormMetricsRegistry.REGISTRY.getMetrics().get(name(INNER_SET, INNER_TIMER)));
-
- assertNotSame(((MetricSet) OUTER.getMetrics().get(INNER_SET)).getMetrics().get(INNER_METER),
- StormMetricsRegistry.REGISTRY.getMetrics().get(name(INNER_SET, INNER_METER)));
- assertSame(existingInnerMeter, StormMetricsRegistry.REGISTRY.getMetrics().get(name(INNER_SET, INNER_METER)));
-
- //Ensure idempotency
- LOG.info("twice register outer set");
- MetricSet newOuter = newMetricSetInstance();
- StormMetricsRegistry.registerMetricSet(newOuter);
- assertSame(OUTER.getMetrics().get(OUTER_TIMER), StormMetricsRegistry.REGISTRY.getMetrics().get(OUTER_TIMER));
- assertSame(OUTER.getMetrics().get(OUTER_METER), StormMetricsRegistry.REGISTRY.getMetrics().get(OUTER_METER));
- assertSame(((MetricSet) OUTER.getMetrics().get(INNER_SET)).getMetrics().get(INNER_TIMER),
- StormMetricsRegistry.REGISTRY.getMetrics().get(name(INNER_SET, INNER_TIMER)));
- assertSame(existingInnerMeter, StormMetricsRegistry.REGISTRY.getMetrics().get(name(INNER_SET, INNER_METER)));
-
- LOG.info("name collision");
- assertThrows(IllegalArgumentException.class, () -> StormMetricsRegistry.registerGauge(name(INNER_SET, INNER_METER), () -> 0));
- }
-
- @Test
- void unregisterMetricSet() {
- StormMetricsRegistry.registerMetricSet(OUTER);
- StormMetricsRegistry.unregisterMetricSet(OUTER);
- assertTrue(StormMetricsRegistry.REGISTRY.getMetrics().isEmpty());
-
- }
-
- private static MetricSet newMetricSetInstance() {
- return new MetricSet() {
- private final MetricSet inner = new MetricSet() {
- private final Map<String, Metric> map = new HashMap<>();
-
- {
- map.put(INNER_METER, new Meter());
- map.put(INNER_TIMER, new Timer());
- }
-
- @Override
- public Map<String, Metric> getMetrics() {
- return map;
- }
- };
- private final Map<String, Metric> outerMap = new HashMap<>();
-
- {
- outerMap.put(OUTER_METER, new Meter());
- outerMap.put(INNER_SET, inner);
- outerMap.put(OUTER_TIMER, new Timer());
- }
-
- @Override
- public Map<String, Metric> getMetrics() {
- return outerMap;
- }
- };
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
index 63df80a..9609c3a 100644
--- a/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
+++ b/storm-server/src/test/java/org/apache/storm/metricstore/rocksdb/RocksDbStoreTest.java
@@ -40,6 +40,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.apache.storm.metric.StormMetricsRegistry;
public class RocksDbStoreTest {
private final static Logger LOG = LoggerFactory.getLogger(RocksDbStoreTest.class);
@@ -57,7 +58,7 @@ public class RocksDbStoreTest {
conf.put(DaemonConfig.STORM_ROCKSDB_CREATE_IF_MISSING, true);
conf.put(DaemonConfig.STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY, 4000);
conf.put(DaemonConfig.STORM_ROCKSDB_METRIC_RETENTION_HOURS, 240);
- store = MetricStoreConfig.configure(conf);
+ store = MetricStoreConfig.configure(conf, new StormMetricsRegistry());
}
@AfterClass
@@ -307,7 +308,7 @@ public class RocksDbStoreTest {
Assert.assertTrue(list.size() >= 2);
// delete anything older than an hour
- MetricsCleaner cleaner = new MetricsCleaner((RocksDbStore)store, 1, 1, null);
+ MetricsCleaner cleaner = new MetricsCleaner((RocksDbStore)store, 1, 1, null, new StormMetricsRegistry());
cleaner.purgeMetrics();
list = getMetricsFromScan(filter);
Assert.assertEquals(1, list.size());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/FaultGenerateUtils.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/FaultGenerateUtils.java b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/FaultGenerateUtils.java
index daf8671..9a34080 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/FaultGenerateUtils.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/FaultGenerateUtils.java
@@ -27,6 +27,8 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
public class FaultGenerateUtils {
@@ -58,6 +60,6 @@ public class FaultGenerateUtils {
} else {
assignment = TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments());
}
- return new Cluster(iNimbus, supervisors, assignment, topologies, config);
+ return new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supervisors, assignment, topologies, config);
}
}
[5/6] storm git commit: STORM-3197: Make StormMetricsRegistry
non-static
Posted by bo...@apache.org.
STORM-3197: Make StormMetricsRegistry non-static
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/8c90f12d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/8c90f12d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/8c90f12d
Branch: refs/heads/master
Commit: 8c90f12dda92eb7b7f93d3d9286ba6035e07127f
Parents: dc70ffd
Author: Stig Rohde Døssing <sr...@apache.org>
Authored: Mon Jul 30 22:53:08 2018 +0200
Committer: Stig Rohde Døssing <sr...@apache.org>
Committed: Mon Sep 17 20:14:16 2018 +0200
----------------------------------------------------------------------
.../storm/cassandra/trident/MapStateTest.java | 7 +-
pom.xml | 2 +-
.../jvm/org/apache/storm/utils/ShellUtils.java | 1 -
.../org/apache/storm/command/KillWorkers.java | 5 +-
.../apache/storm/trident/integration_test.clj | 18 +-
.../test/clj/org/apache/storm/drpc_test.clj | 20 +-
.../test/clj/org/apache/storm/nimbus_test.clj | 21 +-
.../scheduler/multitenant_scheduler_test.clj | 38 ++--
.../clj/org/apache/storm/scheduler_test.clj | 3 +
.../java/org/apache/storm/LocalCluster.java | 21 +-
.../main/java/org/apache/storm/LocalDRPC.java | 14 +-
.../java/org/apache/storm/daemon/drpc/DRPC.java | 22 +-
.../reporters/ConsolePreparableReporter.java | 3 +-
.../reporters/CsvPreparableReporter.java | 3 +-
.../reporters/JmxPreparableReporter.java | 2 +-
.../metrics/reporters/PreparableReporter.java | 4 +-
.../org/apache/storm/daemon/nimbus/Nimbus.java | 227 ++++++++++++-------
.../storm/daemon/supervisor/BasicContainer.java | 22 +-
.../supervisor/BasicContainerLauncher.java | 15 +-
.../storm/daemon/supervisor/Container.java | 154 +++++--------
.../daemon/supervisor/ContainerLauncher.java | 14 +-
.../supervisor/ContainerMemoryTracker.java | 160 +++++++++++++
.../storm/daemon/supervisor/LocalContainer.java | 7 +-
.../supervisor/LocalContainerLauncher.java | 11 +-
.../daemon/supervisor/ReadClusterState.java | 7 +-
.../daemon/supervisor/RunAsUserContainer.java | 15 +-
.../supervisor/RunAsUserContainerLauncher.java | 16 +-
.../apache/storm/daemon/supervisor/Slot.java | 64 +++---
.../storm/daemon/supervisor/SlotMetrics.java | 46 ++++
.../storm/daemon/supervisor/Supervisor.java | 44 +++-
.../apache/storm/localizer/AsyncLocalizer.java | 36 +--
.../storm/localizer/LocalizedResource.java | 5 +-
.../storm/localizer/LocallyCachedBlob.java | 7 +-
.../localizer/LocallyCachedTopologyBlob.java | 9 +-
.../storm/metric/StormMetricsRegistry.java | 104 +++------
.../apache/storm/metricstore/MetricStore.java | 4 +-
.../storm/metricstore/MetricStoreConfig.java | 6 +-
.../metricstore/rocksdb/MetricsCleaner.java | 12 +-
.../storm/metricstore/rocksdb/RocksDbStore.java | 8 +-
.../storm/nimbus/LeaderListenerCallback.java | 10 +-
.../org/apache/storm/pacemaker/Pacemaker.java | 29 ++-
.../org/apache/storm/scheduler/Cluster.java | 17 +-
.../storm/scheduler/SupervisorDetails.java | 1 +
.../apache/storm/scheduler/TopologyDetails.java | 2 -
.../scheduler/blacklist/BlacklistScheduler.java | 6 +-
.../strategies/RasBlacklistStrategy.java | 4 +-
.../storm/scheduler/resource/RAS_Node.java | 2 +-
.../storm/scheduler/resource/ResourceUtils.java | 9 +-
.../normalization/NormalizedResourceOffer.java | 12 +-
.../NormalizedResourceRequest.java | 2 +-
.../normalization/NormalizedResources.java | 12 +-
.../resource/normalization/ResourceMetrics.java | 34 +++
.../scheduling/BaseResourceAwareStrategy.java | 13 +-
.../storm/zookeeper/LeaderElectorImp.java | 9 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 14 +-
.../java/org/apache/storm/PacemakerTest.java | 12 +-
.../org/apache/storm/daemon/drpc/DRPCTest.java | 10 +-
.../daemon/supervisor/BasicContainerTest.java | 43 ++--
.../storm/daemon/supervisor/ContainerTest.java | 12 +-
.../storm/daemon/supervisor/SlotTest.java | 39 ++--
.../storm/localizer/AsyncLocalizerTest.java | 11 +-
.../LocalizedResourceRetentionSetTest.java | 14 +-
.../storm/metric/StormMetricsRegistryTest.java | 111 ---------
.../metricstore/rocksdb/RocksDbStoreTest.java | 5 +-
.../scheduler/blacklist/FaultGenerateUtils.java | 4 +-
.../blacklist/TestBlacklistScheduler.java | 66 +++---
.../resource/TestResourceAwareScheduler.java | 51 +++--
.../storm/scheduler/resource/TestUser.java | 5 +-
.../NormalizedResourceOfferTest.java | 5 +-
.../normalization/NormalizedResourcesTest.java | 9 +-
.../eviction/TestDefaultEvictionStrategy.java | 13 +-
.../TestFIFOSchedulingPriorityStrategy.java | 9 +-
.../TestConstraintSolverStrategy.java | 9 +-
.../TestDefaultResourceAwareStrategy.java | 10 +-
.../TestGenericResourceAwareStrategy.java | 9 +-
.../apache/storm/daemon/drpc/DRPCServer.java | 24 +-
.../daemon/drpc/webapp/DRPCApplication.java | 7 +-
.../storm/daemon/drpc/webapp/DRPCResource.java | 12 +-
.../storm/daemon/logviewer/LogviewerServer.java | 35 +--
.../handler/LogviewerLogDownloadHandler.java | 7 +-
.../handler/LogviewerLogPageHandler.java | 31 ++-
.../handler/LogviewerLogSearchHandler.java | 29 ++-
.../handler/LogviewerProfileHandler.java | 16 +-
.../logviewer/utils/DirectoryCleaner.java | 16 +-
.../logviewer/utils/ExceptionMeterNames.java | 60 +++++
.../daemon/logviewer/utils/ExceptionMeters.java | 66 ------
.../daemon/logviewer/utils/LogCleaner.java | 26 ++-
.../logviewer/utils/LogFileDownloader.java | 15 +-
.../utils/LogviewerResponseBuilder.java | 5 +-
.../daemon/logviewer/utils/WorkerLogs.java | 15 +-
.../logviewer/webapp/LogviewerApplication.java | 22 +-
.../logviewer/webapp/LogviewerResource.java | 76 ++++---
.../org/apache/storm/daemon/ui/UIServer.java | 36 +--
.../daemon/ui/filters/HeaderResponseFilter.java | 9 +-
.../ui/filters/HeaderResponseServletFilter.java | 11 +-
.../daemon/ui/resources/StormApiResource.java | 100 ++++----
.../storm/daemon/drpc/DRPCServerTest.java | 9 +-
.../handler/LogviewerLogPageHandlerTest.java | 4 +-
.../handler/LogviewerLogSearchHandlerTest.java | 7 +-
.../daemon/logviewer/utils/LogCleanerTest.java | 26 ++-
.../daemon/logviewer/utils/WorkerLogsTest.java | 3 +-
101 files changed, 1434 insertions(+), 1033 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/external/storm-cassandra/src/test/java/org/apache/storm/cassandra/trident/MapStateTest.java
----------------------------------------------------------------------
diff --git a/external/storm-cassandra/src/test/java/org/apache/storm/cassandra/trident/MapStateTest.java b/external/storm-cassandra/src/test/java/org/apache/storm/cassandra/trident/MapStateTest.java
index b47165b..9a82564 100644
--- a/external/storm-cassandra/src/test/java/org/apache/storm/cassandra/trident/MapStateTest.java
+++ b/external/storm-cassandra/src/test/java/org/apache/storm/cassandra/trident/MapStateTest.java
@@ -119,15 +119,16 @@ public class MapStateTest {
.persistentAggregate(factory, new Count(), new Fields("state"))
.parallelismHint(1);
- LocalDRPC client = new LocalDRPC();
+ LocalCluster cluster = new LocalCluster();
+ LocalDRPC client = new LocalDRPC(cluster.getMetricRegistry());
+
topology.newDRPCStream("words", client)
.each(new Fields("args"), new Split(), new Fields("word"))
.groupBy(new Fields("word"))
.stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("state"))
.each(new Fields("state"), new FilterNull())
.aggregate(new Fields("state"), new Sum(), new Fields("sum"));
-
- LocalCluster cluster = new LocalCluster();
+
logger.info("Submitting topology.");
cluster.submitTopology("test", new HashMap(), topology.build());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 82cf63f..9564726 100644
--- a/pom.xml
+++ b/pom.xml
@@ -280,7 +280,7 @@
<log4j-over-slf4j.version>1.6.6</log4j-over-slf4j.version>
<log4j.version>2.11.1</log4j.version>
<slf4j.version>1.7.21</slf4j.version>
- <metrics.version>3.1.0</metrics.version>
+ <metrics.version>3.2.6</metrics.version>
<clojure.tools.nrepl.version>0.2.3</clojure.tools.nrepl.version>
<clojure-complete.version>0.2.3</clojure-complete.version>
<mockito.version>2.19.0</mockito.version>
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
index f4df473..d16d00a 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
@@ -26,7 +26,6 @@ import org.apache.storm.Config;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
abstract public class ShellUtils {
public static final Logger LOG = LoggerFactory.getLogger(ShellUtils.class);
public static final OSType osType = getOSType();
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/src/jvm/org/apache/storm/command/KillWorkers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/KillWorkers.java b/storm-core/src/jvm/org/apache/storm/command/KillWorkers.java
index b9afaa5..11affbd 100644
--- a/storm-core/src/jvm/org/apache/storm/command/KillWorkers.java
+++ b/storm-core/src/jvm/org/apache/storm/command/KillWorkers.java
@@ -12,17 +12,16 @@
package org.apache.storm.command;
-import java.io.File;
import java.util.Map;
-import org.apache.storm.Config;
import org.apache.storm.daemon.supervisor.StandaloneSupervisor;
import org.apache.storm.daemon.supervisor.Supervisor;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
public class KillWorkers {
public static void main(String[] args) throws Exception {
Map<String, Object> conf = Utils.readStormConfig();
- try (Supervisor supervisor = new Supervisor(conf, null, new StandaloneSupervisor())) {
+ try (Supervisor supervisor = new Supervisor(conf, null, new StandaloneSupervisor(), new StormMetricsRegistry())) {
supervisor.shutdownAllWorkers(null, null);
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj
index 3f2be2b..e038696 100644
--- a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj
@@ -47,7 +47,7 @@
(deftest test-memory-map-get-tuples
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind feeder (FeederBatchSpout. ["sentence"]))
@@ -74,7 +74,7 @@
(deftest test-word-count
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind feeder (FeederBatchSpout. ["sentence"]))
@@ -107,7 +107,7 @@
;; there's at least one repartitioning after the spout
(deftest test-word-count-committer-spout
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind feeder (FeederCommitterBatchSpout. ["sentence"]))
@@ -146,7 +146,7 @@
(deftest test-count-agg
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(-> topo
@@ -164,7 +164,7 @@
(deftest test-split-merge
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind drpc-stream (-> topo (.newDRPCStream "splitter" drpc)))
@@ -185,7 +185,7 @@
(deftest test-multiple-groupings-same-stream
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind drpc-stream (-> topo (.newDRPCStream "tester" drpc)
@@ -207,7 +207,7 @@
(deftest test-multi-repartition
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind drpc-stream (-> topo (.newDRPCStream "tester" drpc)
@@ -281,7 +281,7 @@
(CombinerAggStateUpdater. (Count.))
(Fields. ["count"])))))
;; test .stateQuery
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(is (thrown? IllegalArgumentException
(-> topo
(.newDRPCStream "words" drpc)
@@ -293,7 +293,7 @@
(deftest test-set-component-resources
(with-open [cluster (LocalCluster. )]
- (with-open [drpc (LocalDRPC.)]
+ (with-open [drpc (LocalDRPC. (.getMetricRegistry cluster))]
(letlocals
(bind topo (TridentTopology.))
(bind feeder (FeederBatchSpout. ["sentence"]))
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/test/clj/org/apache/storm/drpc_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/drpc_test.clj b/storm-core/test/clj/org/apache/storm/drpc_test.clj
index c9b955a..4a1b16f 100644
--- a/storm-core/test/clj/org/apache/storm/drpc_test.clj
+++ b/storm-core/test/clj/org/apache/storm/drpc_test.clj
@@ -41,9 +41,9 @@
)
(deftest test-drpc-flow
- (let [drpc (LocalDRPC.)
+ (let [cluster (LocalCluster.)
+ drpc (LocalDRPC. (.getMetricRegistry cluster))
spout (DRPCSpout. "test" drpc)
- cluster (LocalCluster.)
topology (Thrift/buildTopology
{"1" (Thrift/prepareSpoutDetails spout)}
{"2" (Thrift/prepareBoltDetails
@@ -73,8 +73,8 @@
)
(deftest test-drpc-builder
- (let [drpc (LocalDRPC.)
- cluster (LocalCluster.)
+ (let [cluster (LocalCluster.)
+ drpc (LocalDRPC. (.getMetricRegistry cluster))
builder (LinearDRPCTopologyBuilder. "test")
]
(.addBolt builder exclamation-bolt-drpc 3)
@@ -136,8 +136,8 @@
))
(deftest test-drpc-coordination
- (let [drpc (LocalDRPC.)
- cluster (LocalCluster.)
+ (let [cluster (LocalCluster.)
+ drpc (LocalDRPC. (.getMetricRegistry cluster))
builder (LinearDRPCTopologyBuilder. "square")
]
(.addBolt builder create-tuples 3)
@@ -178,8 +178,8 @@
)))
(deftest test-drpc-coordination-tricky
- (let [drpc (LocalDRPC.)
- cluster (LocalCluster.)
+ (let [cluster (LocalCluster.)
+ drpc (LocalDRPC. (.getMetricRegistry cluster))
builder (LinearDRPCTopologyBuilder. "tricky")
]
(.addBolt builder id-bolt 3)
@@ -210,8 +210,8 @@
)))
(deftest test-drpc-fail-finish
- (let [drpc (LocalDRPC.)
- cluster (LocalCluster.)
+ (let [cluster (LocalCluster.)
+ drpc (LocalDRPC. (.getMetricRegistry cluster))
builder (LinearDRPCTopologyBuilder. "fail2")
]
(.addBolt builder fail-finish-bolt 3)
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index c8bc42a..696cb2b 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -48,6 +48,7 @@
(:import [org.apache.storm.daemon StormCommon])
(:import [org.apache.storm.cluster IStormClusterState StormClusterStateImpl ClusterStateContext ClusterUtils]
[org.apache.storm.assignments LocalAssignmentsBackendFactory])
+ (:import [org.apache.storm.metric StormMetricsRegistry])
(:use [org.apache.storm util daemon-config config log])
(:require [conjure.core])
@@ -61,7 +62,7 @@
([conf inimbus blob-store leader-elector group-mapper cluster-state]
;blacklist scheduler requires nimbus-monitor-freq-secs as input parameter.
(let [conf-with-nimbus-monitor-freq (merge {NIMBUS-MONITOR-FREQ-SECS 10} conf)]
- (Nimbus. conf-with-nimbus-monitor-freq inimbus cluster-state nil blob-store nil leader-elector group-mapper))))
+ (Nimbus. conf-with-nimbus-monitor-freq inimbus cluster-state nil blob-store nil leader-elector group-mapper (StormMetricsRegistry.)))))
(defn- from-json
[^String str]
@@ -1317,7 +1318,7 @@
(with-open [zk (InProcessZookeeper. )]
(with-open [tmp-nimbus-dir (TmpPath.)
_ (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))]
+ (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
(let [nimbus-dir (.getPath tmp-nimbus-dir)]
(letlocals
(bind conf (merge (clojurify-structure (ConfigUtils/readStormConfig))
@@ -1335,7 +1336,7 @@
{}))
(with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. false))))]
+ (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. false))))]
(letlocals
(bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf ass-backend (ClusterStateContext.)))
@@ -1688,7 +1689,7 @@
(with-open [zk (InProcessZookeeper. )]
(with-open [tmp-nimbus-dir (TmpPath.)
_ (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))]
+ (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
(let [nimbus-dir (.getPath tmp-nimbus-dir)]
(letlocals
(bind conf (merge (clojurify-structure (ConfigUtils/readStormConfig))
@@ -1880,8 +1881,8 @@
mock-blob-store (Mockito/mock BlobStore)
conf {NIMBUS-MONITOR-FREQ-SECS 10 NIMBUS-TOPOLOGY-BLOBSTORE-DELETION-DELAY-MS 0}]
(with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))]
- (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil))]
+ (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
+ (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil (StormMetricsRegistry.)))]
(.set (.getHeartbeatsCache nimbus) hb-cache)
(.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (HashSet. inactive-topos))
@@ -1918,8 +1919,8 @@
mock-blob-store (Mockito/mock BlobStore)
conf {NIMBUS-MONITOR-FREQ-SECS 10}]
(with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls] (MockLeaderElector. ))))]
- (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil))]
+ (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
+ (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil (StormMetricsRegistry.)))]
(.set (.getHeartbeatsCache nimbus) hb-cache)
(.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (set inactive-topos))
@@ -1946,7 +1947,7 @@
mock-state (mock-cluster-state)
mock-blob-store (Mockito/mock BlobStore)
mock-tc (Mockito/mock TopoCache)
- nimbus (Nimbus. {NIMBUS-MONITOR-FREQ-SECS 10} nil mock-state nil mock-blob-store mock-tc (MockLeaderElector. ) nil)]
+ nimbus (Nimbus. {NIMBUS-MONITOR-FREQ-SECS 10} nil mock-state nil mock-blob-store mock-tc (MockLeaderElector. ) nil (StormMetricsRegistry.))]
(let [supervisor1-topologies (clojurify-structure (Nimbus/topologiesOnSupervisor assignments "super1"))
user1-topologies (clojurify-structure (.filterAuthorized nimbus "getTopology" supervisor1-topologies))
supervisor2-topologies (clojurify-structure (Nimbus/topologiesOnSupervisor assignments "super2"))
@@ -1967,7 +1968,7 @@
mock-state (mock-cluster-state)
mock-blob-store (Mockito/mock BlobStore)
mock-tc (Mockito/mock TopoCache)
- nimbus (Nimbus. {NIMBUS-MONITOR-FREQ-SECS 10} nil mock-state nil mock-blob-store mock-tc (MockLeaderElector. ) nil)]
+ nimbus (Nimbus. {NIMBUS-MONITOR-FREQ-SECS 10} nil mock-state nil mock-blob-store mock-tc (MockLeaderElector. ) nil (StormMetricsRegistry.))]
(.thenReturn (Mockito/when (.readTopoConf mock-tc (Mockito/eq "authorized") (Mockito/anyObject))) {TOPOLOGY-NAME "authorized"})
(.thenReturn (Mockito/when (.readTopoConf mock-tc (Mockito/eq "topo1") (Mockito/anyObject))) {TOPOLOGY-NAME "topo1"})
(.setAuthorizationHandler nimbus (reify IAuthorizer (permit [this context operation topo-conf] (= "authorized" (get topo-conf TOPOLOGY-NAME)))))
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj
index 910a58a..9e170b9 100644
--- a/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj
+++ b/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj
@@ -20,6 +20,8 @@
(:import [org.apache.storm.daemon.nimbus Nimbus$StandaloneINimbus])
(:import [org.apache.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails
SchedulerAssignmentImpl Topologies TopologyDetails])
+ (:import [org.apache.storm.scheduler.resource.normalization ResourceMetrics])
+ (:import [org.apache.storm.metric StormMetricsRegistry])
(:import [org.apache.storm.scheduler.multitenant Node NodePool FreePool DefaultPool
IsolatedPool MultitenantScheduler]))
@@ -44,7 +46,7 @@
(let [supers (gen-supervisors 5)
topology1 (TopologyDetails. "topology1" {} nil 1, "user")
topology2 (TopologyDetails. "topology2" {} nil 1, "user")
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} (Topologies. {"topology1" topology1 "topology2" topology2}) {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} (Topologies. {"topology1" topology1 "topology2" topology2}) {})
node-map (Node/getAllNodesFrom cluster)]
(is (= 5 (.size node-map)))
(let [node (.get node-map "super0")]
@@ -90,7 +92,7 @@
(deftest test-free-pool
(let [supers (gen-supervisors 5)
topology1 (TopologyDetails. "topology1" {} nil 1, "user")
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} (Topologies. {"topology1" topology1}) {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} (Topologies. {"topology1" topology1}) {})
node-map (Node/getAllNodesFrom cluster)
free-pool (FreePool. )]
;; assign one node so it is not in the pool
@@ -141,7 +143,7 @@
executor2 "bolt1"
executor3 "bolt2"} "user")
topologies (Topologies. {"topology1" topology1})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -179,7 +181,7 @@
executor2 "bolt1"
executor3 "bolt2"} "user")
topologies (Topologies. {"topology1" topology1})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -220,7 +222,7 @@
executor3 "bolt1"
executor4 "bolt1"
executor5 "bolt2"} "user")
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} (Topologies. {"topology1" topology1}) {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} (Topologies. {"topology1" topology1}) {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -261,7 +263,7 @@
executor4 "bolt3"
executor5 "bolt4"} "user")
topologies (Topologies. {"topology1" topology1})
- single-cluster (Cluster. (Nimbus$StandaloneINimbus.) single-super {} topologies {})]
+ single-cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) single-super {} topologies {})]
(let [node-map (Node/getAllNodesFrom single-cluster)
free-pool (FreePool. )
default-pool (DefaultPool. )]
@@ -274,7 +276,7 @@
(is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap single-cluster) "topology1")))
)
- (let [cluster (Cluster. (Nimbus$StandaloneINimbus.) supers (.getAssignments single-cluster) topologies {})
+ (let [cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers (.getAssignments single-cluster) topologies {})
node-map (Node/getAllNodesFrom cluster)
free-pool (FreePool. )
default-pool (DefaultPool. )]
@@ -316,7 +318,7 @@
executor13 "bolt13"
executor14 "bolt14"} "user")
topologies (Topologies. {"topology1" topology1 "topology2" topology2})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -390,7 +392,7 @@
executor3 "bolt2"
executor4 "bolt4"} "user")
topologies (Topologies. {"topology1" topology1})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -435,7 +437,7 @@
executor3 "bolt2"
executor4 "bolt4"} "user")
topologies (Topologies. {"topology1" topology1})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -492,7 +494,7 @@
executor13 "bolt13"
executor14 "bolt14"} "user")
topologies (Topologies. {"topology1" topology1 "topology2" topology2})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -598,7 +600,7 @@
executor13 "bolt13"
executor14 "bolt14"} "user")
topologies (Topologies. {"topology1" topology1 "topology2" topology2})
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)]
;; assign one node so it is not in the pool
(.assign (.get node-map "super0") "topology1" (list executor1) cluster)
@@ -666,7 +668,7 @@
["bolt23" 20 30]
["bolt24" 30 40]]) "userB")
topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers {} topologies {})
node-map (Node/getAllNodesFrom cluster)
conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
scheduler (MultitenantScheduler.)]
@@ -704,7 +706,7 @@
(ExecutorDetails. 15 20) (WorkerSlot. "super0" 1)} nil nil)
}
topologies (Topologies. (to-top-map [topology1]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers existing-assignments topologies {})
node-map (Node/getAllNodesFrom cluster)
conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
scheduler (MultitenantScheduler.)]
@@ -746,7 +748,7 @@
existing-assignments {"topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 1 1) worker-slot-with-multiple-assignments} nil nil)
"topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 2 2) worker-slot-with-multiple-assignments} nil nil)}
topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers existing-assignments topologies {})
conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2 "userB" 1}}
scheduler (MultitenantScheduler.)]
(.prepare scheduler conf)
@@ -773,7 +775,7 @@
(SchedulerAssignmentImpl. "topology1"
{(ExecutorDetails. 0 0) (WorkerSlot. "super0" port-not-reported-by-supervisor)} nil nil)}
topologies (Topologies. (to-top-map [topology1]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers existing-assignments topologies {})
conf {}
scheduler (MultitenantScheduler.)]
(.prepare scheduler conf)
@@ -811,7 +813,7 @@
{(ExecutorDetails. 4 4) worker-slot-with-multiple-assignments
(ExecutorDetails. 5 5) (WorkerSlot. dead-supervisor port-not-reported-by-supervisor)} nil nil)}
topologies (Topologies. (to-top-map [topology1 topology2]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers existing-assignments topologies {})
conf {}
scheduler (MultitenantScheduler.)]
(.prepare scheduler conf)
@@ -849,7 +851,7 @@
(ExecutorDetails. 4 4) (WorkerSlot. "super2" 1)
(ExecutorDetails. 5 5) (WorkerSlot. "super2" 2)} nil nil)}
topologies (Topologies. (to-top-map [topology1]))
- cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments topologies {})
+ cluster (Cluster. (Nimbus$StandaloneINimbus.) (ResourceMetrics. (StormMetricsRegistry.)) supers existing-assignments topologies {})
conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2}}
scheduler (MultitenantScheduler.)]
(.prepare scheduler conf)
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-core/test/clj/org/apache/storm/scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/scheduler_test.clj b/storm-core/test/clj/org/apache/storm/scheduler_test.clj
index f525d5b3..8ab7f5e 100644
--- a/storm-core/test/clj/org/apache/storm/scheduler_test.clj
+++ b/storm-core/test/clj/org/apache/storm/scheduler_test.clj
@@ -19,6 +19,8 @@
(:import [org.apache.storm.scheduler EvenScheduler])
(:import [org.apache.storm.daemon.nimbus Nimbus$StandaloneINimbus])
(:import [org.apache.storm.generated StormTopology])
+ (:import [org.apache.storm.scheduler.resource.normalization ResourceMetrics])
+ (:import [org.apache.storm.metric StormMetricsRegistry])
(:import [org.apache.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails
SchedulerAssignmentImpl Topologies TopologyDetails]))
@@ -128,6 +130,7 @@
assignment2 (SchedulerAssignmentImpl. "topology2" executor->slot2 nil nil)
assignment3 (SchedulerAssignmentImpl. "topology3" executor->slot3 nil nil)
cluster (Cluster. (Nimbus$StandaloneINimbus.)
+ (ResourceMetrics. (StormMetricsRegistry.))
{"supervisor1" supervisor1 "supervisor2" supervisor2}
{"topology1" assignment1 "topology2" assignment2 "topology3" assignment3}
topologies
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/LocalCluster.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/LocalCluster.java b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
index dc8355d..069bfb8 100644
--- a/storm-server/src/main/java/org/apache/storm/LocalCluster.java
+++ b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
@@ -84,6 +84,7 @@ import org.apache.storm.generated.TopologyPageInfo;
import org.apache.storm.generated.WorkerMetrics;
import org.apache.storm.messaging.IContext;
import org.apache.storm.messaging.local.Context;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.scheduler.INimbus;
import org.apache.storm.scheduler.ISupervisor;
@@ -144,6 +145,7 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
private final StormCommonInstaller commonInstaller;
private final SimulatedTime time;
private final NimbusClient.LocalOverride nimbusOverride;
+ private final StormMetricsRegistry metricRegistry;
/**
* Create a default LocalCluster.
@@ -219,6 +221,7 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
this.zookeeper = zookeeper;
conf.putAll(builder.daemonConf);
this.daemonConf = new HashMap<>(conf);
+ this.metricRegistry = new StormMetricsRegistry();
this.portCounter = new AtomicInteger(builder.supervisorSlotPortMin);
ClusterStateContext cs = new ClusterStateContext(DaemonType.NIMBUS, daemonConf);
@@ -232,7 +235,7 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
conf.put(Config.STORM_LOCAL_DIR, nimbusTmp.getPath());
Nimbus nimbus = new Nimbus(conf, builder.inimbus == null ? new StandaloneINimbus() : builder.inimbus,
this.getClusterState(), null, builder.store, builder.topoCache, builder.leaderElector,
- builder.groupMapper);
+ builder.groupMapper, metricRegistry);
if (builder.nimbusWrapper != null) {
nimbus = builder.nimbusWrapper.apply(nimbus);
}
@@ -270,6 +273,8 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
this.nimbusOverride = null;
}
success = true;
+
+ metricRegistry.startMetricsReporters(daemonConf);
} finally {
if (!success) {
close();
@@ -307,7 +312,7 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec) throws Exception {
LOG.info("\n\n\t\tSTARTING LOCAL MODE CLUSTER\n\n");
try (LocalCluster local = new LocalCluster();
- LocalDRPC drpc = new LocalDRPC();
+ LocalDRPC drpc = new LocalDRPC(local.metricRegistry);
DRPCClient.LocalOverride drpcOverride = new DRPCClient.LocalOverride(drpc)) {
T ret = c.call();
@@ -375,6 +380,13 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
}
/**
+ * @return The metrics registry for the local cluster.
+ */
+ public StormMetricsRegistry getMetricRegistry() {
+ return metricRegistry;
+ }
+
+ /**
* @return the base config for the daemons.
*/
public Map<String, Object> getDaemonConf() {
@@ -498,6 +510,9 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
@Override
public synchronized void close() throws Exception {
+ if (metricRegistry != null) {
+ metricRegistry.stopMetricsReporters();
+ }
if (nimbusOverride != null) {
nimbusOverride.close();
}
@@ -648,7 +663,7 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
throw new IllegalArgumentException("Cannot start server in distrubuted mode!");
}
- Supervisor s = new Supervisor(superConf, sharedContext, isuper);
+ Supervisor s = new Supervisor(superConf, sharedContext, isuper, metricRegistry);
s.launch();
s.setLocalNimbus(this.nimbus);
this.nimbus.addSupervisor(s);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/LocalDRPC.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/LocalDRPC.java b/storm-server/src/main/java/org/apache/storm/LocalDRPC.java
index 4efd757..ba243cc 100644
--- a/storm-server/src/main/java/org/apache/storm/LocalDRPC.java
+++ b/storm-server/src/main/java/org/apache/storm/LocalDRPC.java
@@ -24,6 +24,7 @@ import org.apache.storm.daemon.drpc.DRPCThrift;
import org.apache.storm.generated.AuthorizationException;
import org.apache.storm.generated.DRPCExecutionException;
import org.apache.storm.generated.DRPCRequest;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.thrift.TException;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.ServiceRegistry;
@@ -38,9 +39,20 @@ public class LocalDRPC implements ILocalDRPC {
private final DRPC drpc;
private final String serviceId;
+ /**
+ * Creates a LocalDRPC with a default metrics registry.
+ */
public LocalDRPC() {
+ this(new StormMetricsRegistry());
+ }
+
+ /**
+ * Creates a LocalDRPC with the specified metrics registry.
+ * @param metricsRegistry The registry
+ */
+ public LocalDRPC(StormMetricsRegistry metricsRegistry) {
Map<String, Object> conf = ConfigUtils.readStormConfig();
- drpc = new DRPC(conf);
+ drpc = new DRPC(metricsRegistry, conf);
serviceId = ServiceRegistry.registerService(new DRPCThrift(drpc));
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/drpc/DRPC.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/drpc/DRPC.java b/storm-server/src/main/java/org/apache/storm/daemon/drpc/DRPC.java
index 4a91e32..e8edd7d 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/drpc/DRPC.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/drpc/DRPC.java
@@ -52,11 +52,12 @@ public class DRPC implements AutoCloseable {
private static final DRPCExecutionException TIMED_OUT = new WrappedDRPCExecutionException("Timed Out");
private static final DRPCExecutionException SHUT_DOWN = new WrappedDRPCExecutionException("Server Shutting Down");
private static final DRPCExecutionException DEFAULT_FAILED = new WrappedDRPCExecutionException("Request failed");
- private static final Meter meterServerTimedOut = StormMetricsRegistry.registerMeter("drpc:num-server-timedout-requests");
- private static final Meter meterExecuteCalls = StormMetricsRegistry.registerMeter("drpc:num-execute-calls");
- private static final Meter meterResultCalls = StormMetricsRegistry.registerMeter("drpc:num-result-calls");
- private static final Meter meterFailRequestCalls = StormMetricsRegistry.registerMeter("drpc:num-failRequest-calls");
- private static final Meter meterFetchRequestCalls = StormMetricsRegistry.registerMeter("drpc:num-fetchRequest-calls");
+
+ private final Meter meterServerTimedOut;
+ private final Meter meterExecuteCalls;
+ private final Meter meterResultCalls;
+ private final Meter meterFailRequestCalls;
+ private final Meter meterFetchRequestCalls;
static {
TIMED_OUT.set_type(DRPCExceptionType.SERVER_TIMEOUT);
@@ -74,13 +75,18 @@ public class DRPC implements AutoCloseable {
private final AtomicLong _ctr = new AtomicLong(0);
private final IAuthorizer _auth;
- public DRPC(Map<String, Object> conf) {
- this(mkAuthorizationHandler((String) conf.get(DaemonConfig.DRPC_AUTHORIZER), conf),
+ public DRPC(StormMetricsRegistry metricsRegistry, Map<String, Object> conf) {
+ this(metricsRegistry, mkAuthorizationHandler((String) conf.get(DaemonConfig.DRPC_AUTHORIZER), conf),
ObjectReader.getInt(conf.get(DaemonConfig.DRPC_REQUEST_TIMEOUT_SECS), 600) * 1000);
}
- public DRPC(IAuthorizer auth, long timeoutMs) {
+ public DRPC(StormMetricsRegistry metricsRegistry, IAuthorizer auth, long timeoutMs) {
_auth = auth;
+ this.meterServerTimedOut = metricsRegistry.registerMeter("drpc:num-server-timedout-requests");
+ this.meterExecuteCalls = metricsRegistry.registerMeter("drpc:num-execute-calls");
+ this.meterResultCalls = metricsRegistry.registerMeter("drpc:num-result-calls");
+ this.meterFailRequestCalls = metricsRegistry.registerMeter("drpc:num-failRequest-calls");
+ this.meterFetchRequestCalls = metricsRegistry.registerMeter("drpc:num-fetchRequest-calls");
_timer.scheduleAtFixedRate(new TimerTask() {
@Override
public void run() {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java
index 47e4f24..97470ee 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/ConsolePreparableReporter.java
@@ -21,7 +21,7 @@ import org.apache.storm.daemon.metrics.ClientMetricsUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class ConsolePreparableReporter implements PreparableReporter<ConsoleReporter> {
+public class ConsolePreparableReporter implements PreparableReporter {
private static final Logger LOG = LoggerFactory.getLogger(ConsolePreparableReporter.class);
ConsoleReporter reporter = null;
@@ -62,6 +62,7 @@ public class ConsolePreparableReporter implements PreparableReporter<ConsoleRepo
public void stop() {
if (reporter != null) {
LOG.debug("Stopping...");
+ reporter.report();
reporter.stop();
} else {
throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java
index 4952051..51b9112 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/CsvPreparableReporter.java
@@ -23,7 +23,7 @@ import org.apache.storm.daemon.metrics.MetricsUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class CsvPreparableReporter implements PreparableReporter<CsvReporter> {
+public class CsvPreparableReporter implements PreparableReporter {
private static final Logger LOG = LoggerFactory.getLogger(CsvPreparableReporter.class);
CsvReporter reporter = null;
@@ -65,6 +65,7 @@ public class CsvPreparableReporter implements PreparableReporter<CsvReporter> {
public void stop() {
if (reporter != null) {
LOG.debug("Stopping...");
+ reporter.report();
reporter.stop();
} else {
throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java
index 3fc77d1..37e952e 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/JmxPreparableReporter.java
@@ -22,7 +22,7 @@ import org.apache.storm.utils.ObjectReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class JmxPreparableReporter implements PreparableReporter<JmxReporter> {
+public class JmxPreparableReporter implements PreparableReporter {
private static final Logger LOG = LoggerFactory.getLogger(JmxPreparableReporter.class);
JmxReporter reporter = null;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java
index 0e1a6e3..162b8c8 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/metrics/reporters/PreparableReporter.java
@@ -13,12 +13,10 @@
package org.apache.storm.daemon.metrics.reporters;
import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Reporter;
-import java.io.Closeable;
import java.util.Map;
-public interface PreparableReporter<T extends Reporter & Closeable> {
+public interface PreparableReporter {
void prepare(MetricRegistry metricsRegistry, Map<String, Object> topoConf);
void start();
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
index c68aa65..5c48271 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
@@ -170,6 +170,7 @@ import org.apache.storm.scheduler.multitenant.MultitenantScheduler;
import org.apache.storm.scheduler.resource.ResourceAwareScheduler;
import org.apache.storm.scheduler.resource.ResourceUtils;
import org.apache.storm.scheduler.resource.normalization.NormalizedResourceRequest;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
import org.apache.storm.security.INimbusCredentialPlugin;
import org.apache.storm.security.auth.ClientAuthUtils;
import org.apache.storm.security.auth.IAuthorizer;
@@ -225,52 +226,51 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
public static final SimpleVersion MIN_VERSION_SUPPORT_RPC_HEARTBEAT = new SimpleVersion("2.0.0");
private static final Logger LOG = LoggerFactory.getLogger(Nimbus.class);
// Metrics
- private static final Meter submitTopologyWithOptsCalls = StormMetricsRegistry.registerMeter("nimbus:num-submitTopologyWithOpts-calls");
- private static final Meter submitTopologyCalls = StormMetricsRegistry.registerMeter("nimbus:num-submitTopology-calls");
- private static final Meter killTopologyWithOptsCalls = StormMetricsRegistry.registerMeter("nimbus:num-killTopologyWithOpts-calls");
- private static final Meter killTopologyCalls = StormMetricsRegistry.registerMeter("nimbus:num-killTopology-calls");
- private static final Meter rebalanceCalls = StormMetricsRegistry.registerMeter("nimbus:num-rebalance-calls");
- private static final Meter activateCalls = StormMetricsRegistry.registerMeter("nimbus:num-activate-calls");
- private static final Meter deactivateCalls = StormMetricsRegistry.registerMeter("nimbus:num-deactivate-calls");
- private static final Meter debugCalls = StormMetricsRegistry.registerMeter("nimbus:num-debug-calls");
- private static final Meter setWorkerProfilerCalls = StormMetricsRegistry.registerMeter("nimbus:num-setWorkerProfiler-calls");
- private static final Meter getComponentPendingProfileActionsCalls = StormMetricsRegistry.registerMeter(
- "nimbus:num-getComponentPendingProfileActions-calls");
- private static final Meter setLogConfigCalls = StormMetricsRegistry.registerMeter("nimbus:num-setLogConfig-calls");
- private static final Meter uploadNewCredentialsCalls = StormMetricsRegistry.registerMeter("nimbus:num-uploadNewCredentials-calls");
- private static final Meter beginFileUploadCalls = StormMetricsRegistry.registerMeter("nimbus:num-beginFileUpload-calls");
- private static final Meter uploadChunkCalls = StormMetricsRegistry.registerMeter("nimbus:num-uploadChunk-calls");
- private static final Meter finishFileUploadCalls = StormMetricsRegistry.registerMeter("nimbus:num-finishFileUpload-calls");
- private static final Meter downloadChunkCalls = StormMetricsRegistry.registerMeter("nimbus:num-downloadChunk-calls");
- private static final Meter getNimbusConfCalls = StormMetricsRegistry.registerMeter("nimbus:num-getNimbusConf-calls");
- private static final Meter getLogConfigCalls = StormMetricsRegistry.registerMeter("nimbus:num-getLogConfig-calls");
- private static final Meter getTopologyConfCalls = StormMetricsRegistry.registerMeter("nimbus:num-getTopologyConf-calls");
- private static final Meter getTopologyCalls = StormMetricsRegistry.registerMeter("nimbus:num-getTopology-calls");
- private static final Meter getUserTopologyCalls = StormMetricsRegistry.registerMeter("nimbus:num-getUserTopology-calls");
- private static final Meter getClusterInfoCalls = StormMetricsRegistry.registerMeter("nimbus:num-getClusterInfo-calls");
- private static final Meter getLeaderCalls = StormMetricsRegistry.registerMeter("nimbus:num-getLeader-calls");
- private static final Meter isTopologyNameAllowedCalls = StormMetricsRegistry.registerMeter("nimbus:num-isTopologyNameAllowed-calls");
- private static final Meter getTopologyInfoWithOptsCalls = StormMetricsRegistry.registerMeter(
- "nimbus:num-getTopologyInfoWithOpts-calls");
- private static final Meter getTopologyInfoCalls = StormMetricsRegistry.registerMeter("nimbus:num-getTopologyInfo-calls");
- private static final Meter getTopologyPageInfoCalls = StormMetricsRegistry.registerMeter("nimbus:num-getTopologyPageInfo-calls");
- private static final Meter getSupervisorPageInfoCalls = StormMetricsRegistry.registerMeter("nimbus:num-getSupervisorPageInfo-calls");
- private static final Meter getComponentPageInfoCalls = StormMetricsRegistry.registerMeter("nimbus:num-getComponentPageInfo-calls");
- private static final Meter getOwnerResourceSummariesCalls = StormMetricsRegistry.registerMeter(
- "nimbus:num-getOwnerResourceSummaries-calls");
+ private final Meter submitTopologyWithOptsCalls;
+ private final Meter submitTopologyCalls;
+ private final Meter killTopologyWithOptsCalls;
+ private final Meter killTopologyCalls;
+ private final Meter rebalanceCalls;
+ private final Meter activateCalls;
+ private final Meter deactivateCalls;
+ private final Meter debugCalls;
+ private final Meter setWorkerProfilerCalls;
+ private final Meter getComponentPendingProfileActionsCalls;
+ private final Meter setLogConfigCalls;
+ private final Meter uploadNewCredentialsCalls;
+ private final Meter beginFileUploadCalls;
+ private final Meter uploadChunkCalls;
+ private final Meter finishFileUploadCalls;
+ private final Meter beginFileDownloadCalls;
+ private final Meter downloadChunkCalls;
+ private final Meter getNimbusConfCalls;
+ private final Meter getLogConfigCalls;
+ private final Meter getTopologyConfCalls;
+ private final Meter getTopologyCalls;
+ private final Meter getUserTopologyCalls;
+ private final Meter getClusterInfoCalls;
+ private final Meter getLeaderCalls;
+ private final Meter isTopologyNameAllowedCalls;
+ private final Meter getTopologyInfoWithOptsCalls;
+ private final Meter getTopologyInfoCalls;
+ private final Meter getTopologyPageInfoCalls;
+ private final Meter getSupervisorPageInfoCalls;
+ private final Meter getComponentPageInfoCalls;
+ private final Histogram scheduleTopologyTimeMs;
+ private final Meter getOwnerResourceSummariesCalls;
+ private final Meter shutdownCalls;
+ private final Meter processWorkerMetricsCalls;
//Timer
- private static final Timer fileUploadDuration = StormMetricsRegistry.registerTimer("nimbus:files-upload-duration-ms");
- private static final Timer schedulingDuration = StormMetricsRegistry.registerTimer("nimbus:topology-scheduling-duration-ms");
+ private final Timer fileUploadDuration;
+ private final Timer schedulingDuration;
//Scheduler histogram
- private static final Histogram numAddedExecPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-added-executors-per-scheduling");
- private static final Histogram numAddedSlotPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-added-slots-per-scheduling");
- private static final Histogram numRemovedExecPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-removed-executors-per-scheduling");
- private static final Histogram numRemovedSlotPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-removed-slots-per-scheduling");
- private static final Histogram numNetExecIncreasePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-executors-increase-per-scheduling");
- private static final Histogram numNetSlotIncreasePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-slots-increase-per-scheduling");
+ private final Histogram numAddedExecPerScheduling;
+ private final Histogram numAddedSlotPerScheduling;
+ private final Histogram numRemovedExecPerScheduling;
+ private final Histogram numRemovedSlotPerScheduling;
+ private final Histogram numNetExecIncreasePerScheduling;
+ private final Histogram numNetSlotIncreasePerScheduling;
// END Metrics
- private static final Meter shutdownCalls = StormMetricsRegistry.registerMeter("nimbus:num-shutdown-calls");
- private static final Meter processWorkerMetricsCalls = StormMetricsRegistry.registerMeter("nimbus:process-worker-metric-calls");
private static final String STORM_VERSION = VersionInfo.getVersion();
public static List<ACL> getNimbusAcls(Map<String, Object> conf) {
@@ -442,6 +442,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
private final List<ClusterMetricsConsumerExecutor> clusterConsumerExceutors;
private final IGroupMappingServiceProvider groupMapper;
private final IPrincipalToLocal principalToLocal;
+ private final StormMetricsRegistry metricsRegistry;
+ private final ResourceMetrics resourceMetrics;
private MetricStore metricsStore;
private IAuthorizer authorizationHandler;
//Cached CuratorFramework, mainly used for BlobStore.
@@ -451,23 +453,72 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
//May be null if worker tokens are not supported by the thrift transport.
private WorkerTokenManager workerTokenManager;
- public Nimbus(Map<String, Object> conf, INimbus inimbus) throws Exception {
- this(conf, inimbus, null, null, null, null, null);
+ public Nimbus(Map<String, Object> conf, INimbus inimbus, StormMetricsRegistry metricsRegistry) throws Exception {
+ this(conf, inimbus, null, null, null, null, null, metricsRegistry);
}
public Nimbus(Map<String, Object> conf, INimbus inimbus, IStormClusterState stormClusterState, NimbusInfo hostPortInfo,
- BlobStore blobStore, ILeaderElector leaderElector, IGroupMappingServiceProvider groupMapper) throws Exception {
- this(conf, inimbus, stormClusterState, hostPortInfo, blobStore, null, leaderElector, groupMapper);
+ BlobStore blobStore, ILeaderElector leaderElector, IGroupMappingServiceProvider groupMapper,
+ StormMetricsRegistry metricsRegistry) throws Exception {
+ this(conf, inimbus, stormClusterState, hostPortInfo, blobStore, null, leaderElector, groupMapper, metricsRegistry);
}
public Nimbus(Map<String, Object> conf, INimbus inimbus, IStormClusterState stormClusterState, NimbusInfo hostPortInfo,
- BlobStore blobStore, TopoCache topoCache, ILeaderElector leaderElector, IGroupMappingServiceProvider groupMapper)
+ BlobStore blobStore, TopoCache topoCache, ILeaderElector leaderElector, IGroupMappingServiceProvider groupMapper,
+ StormMetricsRegistry metricsRegistry)
throws Exception {
this.conf = conf;
+ this.metricsRegistry = metricsRegistry;
+ this.resourceMetrics = new ResourceMetrics(metricsRegistry);
+ this.submitTopologyWithOptsCalls = metricsRegistry.registerMeter("nimbus:num-submitTopologyWithOpts-calls");
+ this.submitTopologyCalls = metricsRegistry.registerMeter("nimbus:num-submitTopology-calls");
+ this.killTopologyWithOptsCalls = metricsRegistry.registerMeter("nimbus:num-killTopologyWithOpts-calls");
+ this.killTopologyCalls = metricsRegistry.registerMeter("nimbus:num-killTopology-calls");
+ this.rebalanceCalls = metricsRegistry.registerMeter("nimbus:num-rebalance-calls");
+ this.activateCalls = metricsRegistry.registerMeter("nimbus:num-activate-calls");
+ this.deactivateCalls = metricsRegistry.registerMeter("nimbus:num-deactivate-calls");
+ this.debugCalls = metricsRegistry.registerMeter("nimbus:num-debug-calls");
+ this.setWorkerProfilerCalls = metricsRegistry.registerMeter("nimbus:num-setWorkerProfiler-calls");
+ this.getComponentPendingProfileActionsCalls = metricsRegistry.registerMeter(
+ "nimbus:num-getComponentPendingProfileActions-calls");
+ this.setLogConfigCalls = metricsRegistry.registerMeter("nimbus:num-setLogConfig-calls");
+ this.uploadNewCredentialsCalls = metricsRegistry.registerMeter("nimbus:num-uploadNewCredentials-calls");
+ this.beginFileUploadCalls = metricsRegistry.registerMeter("nimbus:num-beginFileUpload-calls");
+ this.uploadChunkCalls = metricsRegistry.registerMeter("nimbus:num-uploadChunk-calls");
+ this.finishFileUploadCalls = metricsRegistry.registerMeter("nimbus:num-finishFileUpload-calls");
+ this.beginFileDownloadCalls = metricsRegistry.registerMeter("nimbus:num-beginFileDownload-calls");
+ this.downloadChunkCalls = metricsRegistry.registerMeter("nimbus:num-downloadChunk-calls");
+ this.getNimbusConfCalls = metricsRegistry.registerMeter("nimbus:num-getNimbusConf-calls");
+ this.getLogConfigCalls = metricsRegistry.registerMeter("nimbus:num-getLogConfig-calls");
+ this.getTopologyConfCalls = metricsRegistry.registerMeter("nimbus:num-getTopologyConf-calls");
+ this.getTopologyCalls = metricsRegistry.registerMeter("nimbus:num-getTopology-calls");
+ this.getUserTopologyCalls = metricsRegistry.registerMeter("nimbus:num-getUserTopology-calls");
+ this.getClusterInfoCalls = metricsRegistry.registerMeter("nimbus:num-getClusterInfo-calls");
+ this.getLeaderCalls = metricsRegistry.registerMeter("nimbus:num-getLeader-calls");
+ this.isTopologyNameAllowedCalls = metricsRegistry.registerMeter("nimbus:num-isTopologyNameAllowed-calls");
+ this.getTopologyInfoWithOptsCalls = metricsRegistry.registerMeter(
+ "nimbus:num-getTopologyInfoWithOpts-calls");
+ this.getTopologyInfoCalls = metricsRegistry.registerMeter("nimbus:num-getTopologyInfo-calls");
+ this.getTopologyPageInfoCalls = metricsRegistry.registerMeter("nimbus:num-getTopologyPageInfo-calls");
+ this.getSupervisorPageInfoCalls = metricsRegistry.registerMeter("nimbus:num-getSupervisorPageInfo-calls");
+ this.getComponentPageInfoCalls = metricsRegistry.registerMeter("nimbus:num-getComponentPageInfo-calls");
+ this.scheduleTopologyTimeMs = metricsRegistry.registerHistogram("nimbus:time-scheduleTopology-ms");
+ this.getOwnerResourceSummariesCalls = metricsRegistry.registerMeter(
+ "nimbus:num-getOwnerResourceSummaries-calls");
+ this.shutdownCalls = metricsRegistry.registerMeter("nimbus:num-shutdown-calls");
+ this.processWorkerMetricsCalls = metricsRegistry.registerMeter("nimbus:process-worker-metric-calls");
+ this.fileUploadDuration = metricsRegistry.registerTimer("nimbus:files-upload-duration-ms");
+ this.schedulingDuration = metricsRegistry.registerTimer("nimbus:topology-scheduling-duration-ms");
+ this.numAddedExecPerScheduling = metricsRegistry.registerHistogram("nimbus:num-added-executors-per-scheduling");
+ this.numAddedSlotPerScheduling = metricsRegistry.registerHistogram("nimbus:num-added-slots-per-scheduling");
+ this.numRemovedExecPerScheduling = metricsRegistry.registerHistogram("nimbus:num-removed-executors-per-scheduling");
+ this.numRemovedSlotPerScheduling = metricsRegistry.registerHistogram("nimbus:num-removed-slots-per-scheduling");
+ this.numNetExecIncreasePerScheduling = metricsRegistry.registerHistogram("nimbus:num-net-executors-increase-per-scheduling");
+ this.numNetSlotIncreasePerScheduling = metricsRegistry.registerHistogram("nimbus:num-net-slots-increase-per-scheduling");
this.metricsStore = null;
try {
- this.metricsStore = MetricStoreConfig.configure(conf);
+ this.metricsStore = MetricStoreConfig.configure(conf, metricsRegistry);
} catch (Exception e) {
// the metrics store is not critical to the operation of the cluster, allow Nimbus to come up
LOG.error("Failed to initialize metric store", e);
@@ -506,7 +557,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
Utils.exitProcess(20, "Error while processing event");
});
this.underlyingScheduler = makeScheduler(conf, inimbus);
- this.scheduler = wrapAsBlacklistScheduler(conf, underlyingScheduler);
+ this.scheduler = wrapAsBlacklistScheduler(conf, underlyingScheduler, metricsRegistry);
this.zkClient = makeZKClient(conf);
this.idToExecutors = new AtomicReference<>(new HashMap<>());
@@ -519,7 +570,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
topoCache = new TopoCache(blobStore, conf);
}
if (leaderElector == null) {
- leaderElector = Zookeeper.zkLeaderElector(conf, zkClient, blobStore, topoCache, stormClusterState, getNimbusAcls(conf));
+ leaderElector = Zookeeper.zkLeaderElector(conf, zkClient, blobStore, topoCache, stormClusterState, getNimbusAcls(conf),
+ metricsRegistry);
}
this.leaderElector = leaderElector;
this.blobStore.setLeaderElector(this.leaderElector);
@@ -578,8 +630,9 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
return ret;
}
- private static IScheduler wrapAsBlacklistScheduler(Map<String, Object> conf, IScheduler scheduler) {
- BlacklistScheduler blacklistWrappedScheduler = new BlacklistScheduler(scheduler);
+ private static IScheduler wrapAsBlacklistScheduler(Map<String, Object> conf, IScheduler scheduler,
+ StormMetricsRegistry metricsRegistry) {
+ BlacklistScheduler blacklistWrappedScheduler = new BlacklistScheduler(scheduler, metricsRegistry);
blacklistWrappedScheduler.prepare(conf);
return blacklistWrappedScheduler;
}
@@ -763,7 +816,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
LOG.info("Assign executors: {}", execToPort.keySet());
numAddedSlot += count;
numAddedExec += execToPort.size();
- }
+ }
} else if (newAssignments.isEmpty()) {
for (Entry<String, Assignment> entry : existingAssignments.entrySet()) {
final Map<List<Long>, NodeInfo> execToPort = entry.getValue().get_executor_node_port();
@@ -772,7 +825,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
LOG.info("Remove executors: {}", execToPort.keySet());
numRemovedSlot += count;
numRemovedExec += execToPort.size();
- }
+ }
} else {
MapDifference<String, Assignment> difference = Maps.difference(existingAssignments, newAssignments);
if (anyChanged = !difference.areEqual()) {
@@ -806,8 +859,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
if (execEntry.getValue().equals(oldExecToSlot.get(execEntry.getKey()))) {
commonExecCount++;
commonSlots.add(execEntry.getValue());
- }
- }
+ }
+ }
long commonSlotCount = commonSlots.size();
//Treat reassign as remove and add
@@ -815,7 +868,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
numRemovedExec += oldExecToSlot.size() - commonExecCount;
numAddedSlot += slots.size() - commonSlotCount;
numAddedExec += execToSlot.size() - commonExecCount;
- }
+ }
}
LOG.debug("{} assignments unchanged: {}", difference.entriesInCommon().size(), difference.entriesInCommon().keySet());
}
@@ -1208,10 +1261,13 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
private static Nimbus launchServer(Map<String, Object> conf, INimbus inimbus) throws Exception {
StormCommon.validateDistributedMode(conf);
validatePortAvailable(conf);
- final Nimbus nimbus = new Nimbus(conf, inimbus);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ final Nimbus nimbus = new Nimbus(conf, inimbus, metricsRegistry);
nimbus.launchServer();
final ThriftServer server = new ThriftServer(conf, new Processor<>(nimbus), ThriftConnectionType.NIMBUS);
+ metricsRegistry.startMetricsReporters(conf);
Utils.addShutdownHookWithDelayedForceKill(() -> {
+ metricsRegistry.stopMetricsReporters();
nimbus.shutdown();
server.stop();
}, 10);
@@ -1654,8 +1710,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
executorsToComponent.put(execDetails, entry.getValue());
}
- return new TopologyDetails(topoId, topoConf, topo, base.get_num_workers(), executorsToComponent, base.get_launch_time_secs(),
- base.get_owner());
+ return new TopologyDetails(topoId, topoConf, topo, base.get_num_workers(), executorsToComponent,
+ base.get_launch_time_secs(), base.get_owner());
}
private void updateHeartbeatsFromZkHeartbeat(String topoId, Set<List<Integer>> allExecutors, Assignment existingAssignment) {
@@ -2049,7 +2105,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
}
Map<String, SupervisorDetails> supervisors =
readAllSupervisorDetails(supervisorToDeadPorts, topologies, missingAssignmentTopologies);
- Cluster cluster = new Cluster(inimbus, supervisors, topoToSchedAssignment, topologies, conf);
+ Cluster cluster = new Cluster(inimbus, resourceMetrics, supervisors, topoToSchedAssignment, topologies, conf);
cluster.setStatusMap(idToSchedStatus.get());
schedulingStartTimeNs.set(Time.nanoTime());
@@ -2221,9 +2277,9 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
execToNodePort = new HashMap<>();
}
Set<String> allNodes = new HashSet<>();
- for (List<Object> nodePort : execToNodePort.values()) {
- allNodes.add((String) nodePort.get(0));
- }
+ for (List<Object> nodePort : execToNodePort.values()) {
+ allNodes.add((String) nodePort.get(0));
+ }
Map<String, String> allNodeHost = new HashMap<>();
Assignment existingAssignment = existingAssignments.get(topoId);
if (existingAssignment != null) {
@@ -2866,26 +2922,22 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
}
});
- StormMetricsRegistry.registerGauge("nimbus:num-supervisors", () -> state.supervisors(null).size());
- StormMetricsRegistry.registerGauge("nimbus:fragmented-memory", this::fragmentedMemory);
- StormMetricsRegistry.registerGauge("nimbus:fragmented-cpu", this::fragmentedCpu);
- StormMetricsRegistry.registerGauge("nimbus:available-memory", () -> nodeIdToResources.get().values()
- .parallelStream()
+ metricsRegistry.registerGauge("nimbus:num-supervisors", () -> state.supervisors(null).size());
+ metricsRegistry.registerGauge("nimbus:fragmented-memory", this::fragmentedMemory);
+ metricsRegistry.registerGauge("nimbus:fragmented-cpu", this::fragmentedCpu);
+ metricsRegistry.registerGauge("nimbus:available-memory", () -> nodeIdToResources.get().values().parallelStream()
.mapToDouble(SupervisorResources::getAvailableMem)
.sum());
- StormMetricsRegistry.registerGauge("nimbus:available-cpu", () -> nodeIdToResources.get().values()
- .parallelStream()
+ metricsRegistry.registerGauge("nimbus:available-cpu", () -> nodeIdToResources.get().values().parallelStream()
.mapToDouble(SupervisorResources::getAvailableCpu)
.sum());
- StormMetricsRegistry.registerGauge("nimbus:total-memory", () -> nodeIdToResources.get().values()
- .parallelStream()
+ metricsRegistry.registerGauge("nimbus:total-memory", () -> nodeIdToResources.get().values().parallelStream()
.mapToDouble(SupervisorResources::getTotalMem)
.sum());
- StormMetricsRegistry.registerGauge("nimbus:total-cpu", () -> nodeIdToResources.get().values()
- .parallelStream()
+ metricsRegistry.registerGauge("nimbus:total-cpu", () -> nodeIdToResources.get().values().parallelStream()
.mapToDouble(SupervisorResources::getTotalCpu)
.sum());
- StormMetricsRegistry.registerGauge("nimbus:longest-scheduling-time-ms", () -> {
+ metricsRegistry.registerGauge("nimbus:longest-scheduling-time-ms", () -> {
//We want to update longest scheduling time in real time in case scheduler get stuck
// Get current time before startTime to avoid potential race with scheduler's Timer
Long currTime = Time.nanoTime();
@@ -2893,19 +2945,18 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
return TimeUnit.NANOSECONDS.toMillis(startTime == null ?
longestSchedulingTime.get() : Math.max(currTime - startTime, longestSchedulingTime.get()));
});
- StormMetricsRegistry.registerMeter("nimbus:num-launched").mark();
- StormMetricsRegistry.startMetricsReporters(conf);
+ metricsRegistry.registerMeter("nimbus:num-launched").mark();
- timer.scheduleRecurring(0, ObjectReader.getInt(conf.get(DaemonConfig.STORM_CLUSTER_METRICS_CONSUMER_PUBLISH_INTERVAL_SECS)),
- () -> {
- try {
- if (isLeader()) {
- sendClusterMetricsToExecutors();
+ timer.scheduleRecurring(0, ObjectReader.getInt(conf.get(DaemonConfig.STORM_CLUSTER_METRICS_CONSUMER_PUBLISH_INTERVAL_SECS)),
+ () -> {
+ try {
+ if (isLeader()) {
+ sendClusterMetricsToExecutors();
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
}
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- });
+ });
} catch (Exception e) {
if (Utils.exceptionCauseIsInstanceOf(InterruptedException.class, e)) {
throw e;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
index 2a5e4f2..1e61be6 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java
@@ -58,6 +58,8 @@ import org.slf4j.LoggerFactory;
import static org.apache.storm.daemon.nimbus.Nimbus.MIN_VERSION_SUPPORT_RPC_HEARTBEAT;
import static org.apache.storm.utils.Utils.OR;
+import org.apache.storm.metric.StormMetricsRegistry;
+
/**
* A container that runs processes on the local box.
*/
@@ -91,12 +93,15 @@ public class BasicContainer extends Container {
* @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
* @param localState the local state of the supervisor. May be null if partial recovery
* @param workerId the id of the worker to use. Must not be null if doing a partial recovery.
+ * @param metricsRegistry The metrics registry.
+ * @param containerMemoryTracker The shared memory tracker for the supervisor's containers
*/
public BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort,
int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- LocalState localState, String workerId) throws IOException {
+ LocalState localState, String workerId, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
this(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, localState,
- workerId, null, null, null);
+ workerId, metricsRegistry, containerMemoryTracker, null, null, null);
}
/**
@@ -111,18 +116,21 @@ public class BasicContainer extends Container {
* @param resourceIsolationManager used to isolate resources for a container can be null if no isolation is used.
* @param localState the local state of the supervisor. May be null if partial recovery
* @param workerId the id of the worker to use. Must not be null if doing a partial recovery.
+ * @param metricsRegistry The metrics registry.
+ * @param containerMemoryTracker The shared memory tracker for the supervisor's containers
* @param ops file system operations (mostly for testing) if null a new one is made
* @param topoConf the config of the topology (mostly for testing) if null and not a partial recovery the real conf is
* read.
- * @param profileCmd the command to use when profiling (used for testing)
+ * @param profileCmd the command to use when profiling (used for testing)
* @throws IOException on any error
* @throws ContainerRecoveryException if the Container could not be recovered.
*/
BasicContainer(ContainerType type, Map<String, Object> conf, String supervisorId, int supervisorPort, int port,
- LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
- LocalState localState, String workerId, Map<String, Object> topoConf,
- AdvancedFSOps ops, String profileCmd) throws IOException {
- super(type, conf, supervisorId, supervisorPort, port, assignment, resourceIsolationManager, workerId, topoConf, ops);
+ LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager, LocalState localState, String workerId,
+ StormMetricsRegistry metricsRegistry, ContainerMemoryTracker containerMemoryTracker, Map<String, Object> topoConf,
+ AdvancedFSOps ops, String profileCmd) throws IOException {
+ super(type, conf, supervisorId, supervisorPort, port, assignment,
+ resourceIsolationManager, workerId, topoConf, ops, metricsRegistry, containerMemoryTracker);
assert (localState != null);
_localState = localState;
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java
index 715fd61..9c000cb 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainerLauncher.java
@@ -17,6 +17,7 @@ import java.util.Map;
import org.apache.storm.container.ResourceIsolationInterface;
import org.apache.storm.daemon.supervisor.Container.ContainerType;
import org.apache.storm.generated.LocalAssignment;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.LocalState;
/**
@@ -27,19 +28,25 @@ public class BasicContainerLauncher extends ContainerLauncher {
private final Map<String, Object> _conf;
private final String _supervisorId;
private final int _supervisorPort;
+ private final StormMetricsRegistry metricsRegistry;
+ private final ContainerMemoryTracker containerMemoryTracker;
public BasicContainerLauncher(Map<String, Object> conf, String supervisorId, int supervisorPort,
- ResourceIsolationInterface resourceIsolationManager) throws IOException {
+ ResourceIsolationInterface resourceIsolationManager, StormMetricsRegistry metricsRegistry,
+ ContainerMemoryTracker containerMemoryTracker) throws IOException {
_conf = conf;
_supervisorId = supervisorId;
_supervisorPort = supervisorPort;
_resourceIsolationManager = resourceIsolationManager;
+ this.metricsRegistry = metricsRegistry;
+ this.containerMemoryTracker = containerMemoryTracker;
}
@Override
public Container launchContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
Container container = new BasicContainer(ContainerType.LAUNCH, _conf, _supervisorId, _supervisorPort, port,
- assignment, _resourceIsolationManager, state, null);
+ assignment, _resourceIsolationManager, state, null, metricsRegistry,
+ containerMemoryTracker);
container.setup();
container.launch();
return container;
@@ -48,12 +55,12 @@ public class BasicContainerLauncher extends ContainerLauncher {
@Override
public Container recoverContainer(int port, LocalAssignment assignment, LocalState state) throws IOException {
return new BasicContainer(ContainerType.RECOVER_FULL, _conf, _supervisorId, _supervisorPort, port, assignment,
- _resourceIsolationManager, state, null);
+ _resourceIsolationManager, state, null, metricsRegistry, containerMemoryTracker);
}
@Override
public Killable recoverContainer(String workerId, LocalState localState) throws IOException {
return new BasicContainer(ContainerType.RECOVER_PARTIAL, _conf, _supervisorId, _supervisorPort, -1, null,
- _resourceIsolationManager, localState, workerId);
+ _resourceIsolationManager, localState, workerId, metricsRegistry, containerMemoryTracker);
}
}
[2/6] storm git commit: STORM-3197: Make StormMetricsRegistry
non-static
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
index 6cf8a0e..7815154 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/blacklist/TestBlacklistScheduler.java
@@ -42,6 +42,8 @@ import java.util.Map;
import java.util.Set;
import java.util.List;
import java.util.ArrayList;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
public class TestBlacklistScheduler {
@@ -67,15 +69,17 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+ Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<>(), topologies, config);
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), metricsRegistry);
bs.prepare(config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
Assert.assertEquals("blacklist", Collections.singleton("host-0"), cluster.getBlacklistedHosts());
}
@@ -98,16 +102,18 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+ Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), metricsRegistry);
bs.prepare(config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap, "sup-0", 0), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap, "sup-0", 0), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap, "sup-0", 0), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap, "sup-0", 0), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
bs.schedule(topologies, cluster);
Assert.assertEquals("blacklist", Collections.singleton("host-0"), cluster.getBlacklistedHosts());
}
@@ -130,15 +136,17 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+ Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), metricsRegistry);
bs.prepare(config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
bs.schedule(topologies, cluster);
Assert.assertEquals("blacklist", Collections.singleton("host-0"), cluster.getBlacklistedHosts());
for (int i = 0; i < 300 / 10 - 2; i++) {
@@ -170,22 +178,24 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+ Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), metricsRegistry);
bs.prepare(config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap, "sup-0"), TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
- cluster = new Cluster(iNimbus, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
Assert.assertEquals("blacklist", Collections.singleton("host-0"), cluster.getBlacklistedHosts());
topoMap.put(topo2.getId(), topo2);
topoMap.put(topo3.getId(), topo3);
topoMap.put(topo4.getId(), topo4);
topologies = new Topologies(topoMap);
- cluster = new Cluster(iNimbus, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
bs.schedule(topologies, cluster);
Assert.assertEquals("blacklist", Collections.emptySet(), cluster.getBlacklistedHosts());
}
@@ -205,7 +215,7 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
topoMap.put(topo2.getId(), topo2);
Topologies topologies = new Topologies(topoMap);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), new StormMetricsRegistry());
bs.prepare(config);
List<Map<Integer, List<Integer>>> faultList = new ArrayList<>();
@@ -292,11 +302,13 @@ public class TestBlacklistScheduler {
topoMap.put(topo1.getId(), topo1);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
- BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler());
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ ResourceMetrics resourceMetrics = new ResourceMetrics(metricsRegistry);
+ Cluster cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ BlacklistScheduler bs = new BlacklistScheduler(new DefaultScheduler(), metricsRegistry);
bs.prepare(config);
bs.schedule(topologies,cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap,"sup-0"),TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removeSupervisorFromSupervisors(supMap,"sup-0"),TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
for (int i = 0 ; i < 20 ; i++){
bs.schedule(topologies,cluster);
}
@@ -304,9 +316,9 @@ public class TestBlacklistScheduler {
cached.add("sup-1");
cached.add("sup-2");
Assert.assertEquals(cached,bs.cachedSupervisors.keySet());
- cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
bs.schedule(topologies,cluster);
- cluster = new Cluster(iNimbus, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap,"sup-0",0),TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
+ cluster = new Cluster(iNimbus, resourceMetrics, TestUtilsForBlacklistScheduler.removePortFromSupervisors(supMap,"sup-0",0),TestUtilsForBlacklistScheduler.assignmentMapToImpl(cluster.getAssignments()), topologies, config);
for (int i = 0 ;i < 20 ; i++){
bs.schedule(topologies, cluster);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
index 74d3ae3..7e118d4 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestResourceAwareScheduler.java
@@ -62,6 +62,9 @@ import org.slf4j.LoggerFactory;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.*;
import static org.junit.Assert.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestResourceAwareScheduler {
private static final Logger LOG = LoggerFactory.getLogger(TestResourceAwareScheduler.class);
@@ -86,7 +89,7 @@ public class TestResourceAwareScheduler {
TopologyDetails topology1 = genTopology("topology1", config, 1, 0, 2, 0, 0, 0, "user");
TopologyDetails topology2 = genTopology("topology2", config, 1, 0, 2, 0, 0, 0, "user");
Topologies topologies = new Topologies(topology1, topology2);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
Map<String, RAS_Node> nodes = RAS_Nodes.getAllNodesFrom(cluster);
assertEquals(5, nodes.size());
RAS_Node node = nodes.get("r000s000");
@@ -156,7 +159,7 @@ public class TestResourceAwareScheduler {
TopologyDetails topology1 = genTopology("topology1", config, 1, 1, 1, 1, 0, 0, "user");
Topologies topologies = new Topologies(topology1);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -205,7 +208,7 @@ public class TestResourceAwareScheduler {
ResourceAwareScheduler rs = new ResourceAwareScheduler();
Topologies topologies = new Topologies(topology1, topology2);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -254,7 +257,7 @@ public class TestResourceAwareScheduler {
ResourceAwareScheduler rs = new ResourceAwareScheduler();
Topologies topologies = new Topologies(topology1);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -298,7 +301,7 @@ public class TestResourceAwareScheduler {
ResourceAwareScheduler rs = new ResourceAwareScheduler();
Topologies topologies = new Topologies(topology1);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -394,7 +397,7 @@ public class TestResourceAwareScheduler {
// Test1: When a worker fails, RAS does not alter existing assignments on healthy workers
ResourceAwareScheduler rs = new ResourceAwareScheduler();
Topologies topologies = new Topologies(topology2);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.prepare(config1);
rs.schedule(topologies, cluster);
@@ -439,7 +442,7 @@ public class TestResourceAwareScheduler {
supMap1.remove("r000s000"); // mock the supervisor r000s000 as a failed supervisor
topologies = new Topologies(topology1);
- Cluster cluster1 = new Cluster(iNimbus, supMap1, existingAssignments, topologies, config1);
+ Cluster cluster1 = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap1, existingAssignments, topologies, config1);
rs.schedule(topologies, cluster1);
newAssignment = cluster1.getAssignmentById(topology1.getId());
@@ -468,7 +471,7 @@ public class TestResourceAwareScheduler {
supMap1.remove("r000s000"); // mock the supervisor r000s000 as a failed supervisor
topologies = new Topologies(topology1);
- cluster1 = new Cluster(iNimbus, supMap1, existingAssignments, topologies, config1);
+ cluster1 = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap1, existingAssignments, topologies, config1);
rs.schedule(topologies, cluster1);
newAssignment = cluster1.getAssignmentById(topology1.getId());
@@ -482,14 +485,14 @@ public class TestResourceAwareScheduler {
// Test4: Scheduling a new topology does not disturb other assignments unnecessarily
topologies = new Topologies(topology1);
- cluster1 = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ cluster1 = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.schedule(topologies, cluster1);
assignment = cluster1.getAssignmentById(topology1.getId());
executorToSlot = assignment.getExecutorToSlot();
copyOfOldMapping = new HashMap<>(executorToSlot);
topologies = addTopologies(topologies, topology2);
- cluster1 = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ cluster1 = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.schedule(topologies, cluster1);
newAssignment = cluster1.getAssignmentById(topology1.getId());
@@ -575,7 +578,7 @@ public class TestResourceAwareScheduler {
ResourceAwareScheduler rs = new ResourceAwareScheduler();
LOG.info("\n\n\t\tScheduling topologies 1, 2 and 3");
Topologies topologies = new Topologies(topology1, topology2, topology3);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.prepare(config1);
rs.schedule(topologies, cluster);
@@ -602,7 +605,7 @@ public class TestResourceAwareScheduler {
// Test2: Launch topo 1, 2 and 4, they together request a little more mem than available, so one of the 3 topos will not be
// scheduled
topologies = new Topologies(topology1, topology2, topology4);
- cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.prepare(config1);
rs.schedule(topologies, cluster);
int numTopologiesAssigned = 0;
@@ -624,7 +627,7 @@ public class TestResourceAwareScheduler {
LOG.info("\n\n\t\tScheduling just topo 5");
//Test3: "Launch topo5 only, both mem and cpu should be exactly used up"
topologies = new Topologies(topology5);
- cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.prepare(config1);
rs.schedule(topologies, cluster);
superToCpu = getSupervisorToCpuUsage(cluster, topologies);
@@ -668,7 +671,7 @@ public class TestResourceAwareScheduler {
TopologyDetails topology1 = new TopologyDetails("topology1", config1, stormTopology1, 1, executorMap1, 0, "user");
ResourceAwareScheduler rs = new ResourceAwareScheduler();
Topologies topologies = new Topologies(topology1);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config1);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config1);
rs.prepare(config1);
rs.schedule(topologies, cluster);
assertEquals("Running - Fully Scheduled by DefaultResourceAwareStrategy", cluster.getStatusMap().get(topology1.getId()));
@@ -688,7 +691,7 @@ public class TestResourceAwareScheduler {
Map<ExecutorDetails, String> executorMap2 = genExecsAndComps(stormTopology2);
TopologyDetails topology2 = new TopologyDetails("topology2", config2, stormTopology2, 1, executorMap2, 0, "user");
topologies = new Topologies(topology2);
- cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config2);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config2);
rs.prepare(config2);
rs.schedule(topologies, cluster);
String status = cluster.getStatusMap().get(topology2.getId());
@@ -719,7 +722,7 @@ public class TestResourceAwareScheduler {
genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -755,7 +758,7 @@ public class TestResourceAwareScheduler {
genTopology("topo-13", config, 5, 15, 1, 1, currentTime - 16, 29, "derek"),
genTopology("topo-14", config, 5, 15, 1, 1, currentTime - 16, 20, "derek"),
genTopology("topo-15", config, 5, 15, 1, 1, currentTime - 24, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -779,7 +782,7 @@ public class TestResourceAwareScheduler {
Topologies topologies = new Topologies(
genTopology("topo-1", config, 5, 15, 1, 1, currentTime - 2, 20, "jerry"),
genTopology("topo-2", config, 5, 15, 1, 1, currentTime - 8, 29, "jerry"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -810,7 +813,7 @@ public class TestResourceAwareScheduler {
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 10, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -838,7 +841,7 @@ public class TestResourceAwareScheduler {
}
Map<String, String> statusMap = cluster.getStatusMap();
LOG.warn("Rescheduling with removed Supervisor....");
- cluster = new Cluster(iNimbus, supMap, newAssignments, topologies, config);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, newAssignments, topologies, config);
cluster.setStatusMap(statusMap);
rs.schedule(topologies, cluster);
@@ -858,7 +861,7 @@ public class TestResourceAwareScheduler {
Topologies topologies = new Topologies(
genTopology("topo-1", config, 1, 0, 2, 0, currentTime - 2, 29, "user"),
genTopology("topo-2", config, 1, 0, 2, 0, currentTime - 2, 10, "user"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -900,7 +903,7 @@ public class TestResourceAwareScheduler {
TopologyDetails topo3 = genTopology("topo-3", config, 1, 2, 1, 1, currentTime - 2, 20, "jerry");
Topologies topologies = new Topologies(topo1, topo2, topo3);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -941,7 +944,7 @@ public class TestResourceAwareScheduler {
0, genExecsAndComps(stormTopology), 0, "jerry");
Topologies topologies = new Topologies(topo);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -1026,7 +1029,7 @@ public class TestResourceAwareScheduler {
}
}
Topologies topologies = new Topologies(topologyDetailsMap);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
long startTime = Time.currentTimeMillis();
rs.prepare(config);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
index 2876c7d..b71b9a6 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/TestUser.java
@@ -35,6 +35,9 @@ import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareSched
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.userRes;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.userResourcePool;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestUser {
private static final Logger LOG = LoggerFactory.getLogger(TestUser.class);
@@ -50,7 +53,7 @@ public class TestUser {
TopologyDetails topo1 = genTopology("topo-1", config, 1, 1, 2, 1, Time.currentTimeSecs() - 24, 9, "user1");
Topologies topologies = new Topologies(topo1);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
User user1 = new User("user1", toDouble(resourceUserPool.get("user1")));
WorkerSlot slot = cluster.getAvailableSlots().get(0);
cluster.assign(slot, topo1.getId(), topo1.getExecutors());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOfferTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOfferTest.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOfferTest.java
index ec13632..f75ac03 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOfferTest.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourceOfferTest.java
@@ -21,6 +21,7 @@ package org.apache.storm.scheduler.resource.normalization;
import java.util.HashMap;
import java.util.Map;
import org.apache.storm.Constants;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.junit.Assert;
import org.junit.Test;
@@ -29,7 +30,7 @@ public class NormalizedResourceOfferTest {
public void testNodeOverExtendedCpu() {
NormalizedResourceOffer availableResources = createOffer(100.0, 0.0);
NormalizedResourceOffer scheduledResources = createOffer(110.0, 0.0);
- availableResources.remove(scheduledResources);
+ availableResources.remove(scheduledResources, new ResourceMetrics(new StormMetricsRegistry()));
Assert.assertEquals(0.0, availableResources.getTotalCpu(), 0.001);
}
@@ -37,7 +38,7 @@ public class NormalizedResourceOfferTest {
public void testNodeOverExtendedMemory() {
NormalizedResourceOffer availableResources = createOffer(0.0, 5.0);
NormalizedResourceOffer scheduledResources = createOffer(0.0, 10.0);
- availableResources.remove(scheduledResources);
+ availableResources.remove(scheduledResources, new ResourceMetrics(new StormMetricsRegistry()));
Assert.assertEquals(0.0, availableResources.getTotalMemoryMb(), 0.001);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourcesTest.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourcesTest.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourcesTest.java
index 3c68bb8..15113fc 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourcesTest.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/normalization/NormalizedResourcesTest.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.function.BiConsumer;
import org.apache.storm.Constants;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
@@ -94,7 +95,7 @@ public class NormalizedResourcesTest {
NormalizedResources resources = new NormalizedResources(normalize(Collections.singletonMap(gpuResourceName, 1)));
NormalizedResources removedResources = new NormalizedResources(normalize(Collections.singletonMap(gpuResourceName, 2)));
- resources.remove(removedResources);
+ resources.remove(removedResources, new ResourceMetrics(new StormMetricsRegistry()));
Map<String, Double> normalizedMap = resources.toNormalizedMap();
assertThat(normalizedMap.get(gpuResourceName), is(0.0));
}
@@ -104,7 +105,7 @@ public class NormalizedResourcesTest {
NormalizedResources resources = new NormalizedResources(normalize(Collections.singletonMap(Constants.COMMON_CPU_RESOURCE_NAME, 1)));
NormalizedResources removedResources = new NormalizedResources(normalize(Collections.singletonMap(Constants.COMMON_CPU_RESOURCE_NAME, 2)));
- resources.remove(removedResources);
+ resources.remove(removedResources, new ResourceMetrics(new StormMetricsRegistry()));
assertThat(resources.getTotalCpu(), is(0.0));
}
@@ -113,7 +114,7 @@ public class NormalizedResourcesTest {
NormalizedResources resources = new NormalizedResources(normalize(Collections.singletonMap(Constants.COMMON_CPU_RESOURCE_NAME, 2)));
NormalizedResources removedResources = new NormalizedResources(normalize(Collections.singletonMap(Constants.COMMON_CPU_RESOURCE_NAME, 1)));
- resources.remove(removedResources);
+ resources.remove(removedResources, new ResourceMetrics(new StormMetricsRegistry()));
Map<String, Double> normalizedMap = resources.toNormalizedMap();
assertThat(normalizedMap.get(Constants.COMMON_CPU_RESOURCE_NAME), is(1.0));
@@ -125,7 +126,7 @@ public class NormalizedResourcesTest {
NormalizedResources resources = new NormalizedResources(normalize(Collections.singletonMap(gpuResourceName, 15)));
NormalizedResources removedResources = new NormalizedResources(normalize(Collections.singletonMap(gpuResourceName, 1)));
- resources.remove(removedResources);
+ resources.remove(removedResources, new ResourceMetrics(new StormMetricsRegistry()));
Map<String, Double> normalizedMap = resources.toNormalizedMap();
assertThat(normalizedMap.get(gpuResourceName), is(14.0));
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
index 8e66af5..f8c2aeb 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/eviction/TestDefaultEvictionStrategy.java
@@ -33,6 +33,9 @@ import java.util.Map;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestDefaultEvictionStrategy {
private static final Logger LOG = LoggerFactory.getLogger(TestDefaultEvictionStrategy.class);
private static int currentTime = 1450418597;
@@ -57,7 +60,7 @@ public class TestDefaultEvictionStrategy {
genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -89,7 +92,7 @@ public class TestDefaultEvictionStrategy {
genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
LOG.info("\n\n\t\tScheduling topos 2 to 5...");
@@ -123,7 +126,7 @@ public class TestDefaultEvictionStrategy {
genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 20, "bobby"),
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 15, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
rs.schedule(topologies, cluster);
@@ -179,7 +182,7 @@ public class TestDefaultEvictionStrategy {
genTopology("topo-2", config, 1, 0, 1, 0, currentTime - 2, 20, "jerry"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
genTopology("topo-6", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
LOG.info("\n\n\t\tScheduling topos 1,2,5,6");
@@ -231,7 +234,7 @@ public class TestDefaultEvictionStrategy {
genTopology("topo-3", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
genTopology("topo-4", config, 1, 0, 1, 0, currentTime - 2, 10, "bobby"),
genTopology("topo-5", config, 1, 0, 1, 0, currentTime - 2, 29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
LOG.info("\n\n\t\tScheduling topos 1,3,4,5");
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
index 47e6ea0..472ea5e 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/priority/TestFIFOSchedulingPriorityStrategy.java
@@ -35,6 +35,9 @@ import org.slf4j.LoggerFactory;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestFIFOSchedulingPriorityStrategy {
private static final Logger LOG = LoggerFactory.getLogger(TestFIFOSchedulingPriorityStrategy.class);
@@ -53,7 +56,7 @@ public class TestFIFOSchedulingPriorityStrategy {
genTopology("topo-2-bobby", config, 1, 0, 1, 0,Time.currentTimeSecs() - 200,10, "bobby"),
genTopology("topo-3-bobby", config, 1, 0, 1, 0,Time.currentTimeSecs() - 300,20, "bobby"),
genTopology("topo-4-derek", config, 1, 0, 1, 0,Time.currentTimeSecs() - 201,29, "derek"));
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
@@ -67,7 +70,7 @@ public class TestFIFOSchedulingPriorityStrategy {
topologies = addTopologies(topologies,
genTopology("topo-5-derek", config, 1, 0, 1, 0,Time.currentTimeSecs() - 15,29, "derek"));
- cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.schedule(topologies, cluster);
assertTopologiesFullyScheduled(cluster, "topo-1-jerry", "topo-2-bobby", "topo-4-derek", "topo-5-derek");
@@ -79,7 +82,7 @@ public class TestFIFOSchedulingPriorityStrategy {
topologies = addTopologies(topologies,
genTopology("topo-6-bobby", config, 1, 0, 1, 0,Time.currentTimeSecs() - 10,29, "bobby"));
- cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.schedule(topologies, cluster);
assertTopologiesFullyScheduled(cluster, "topo-1-jerry", "topo-2-bobby", "topo-5-derek", "topo-6-bobby");
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestConstraintSolverStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestConstraintSolverStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestConstraintSolverStrategy.java
index 1c6e153..df3afba 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestConstraintSolverStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestConstraintSolverStrategy.java
@@ -49,6 +49,9 @@ import java.util.Map;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestConstraintSolverStrategy {
private static final Logger LOG = LoggerFactory.getLogger(TestConstraintSolverStrategy.class);
private static final int MAX_TRAVERSAL_DEPTH = 2000;
@@ -86,7 +89,7 @@ public class TestConstraintSolverStrategy {
public Cluster makeCluster(TopologyDetails topo) {
Topologies topologies = new Topologies(topo);
Map<String, SupervisorDetails> supMap = genSupervisors(4, 2, 120, 1200);
- return new Cluster(new INimbusTest(), supMap, new HashMap<>(), topologies, new Config());
+ return new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, new Config());
}
public void basicUnitTestWithKillAndRecover(ConstraintSolverStrategy cs, int boltParallel) {
@@ -214,10 +217,10 @@ public class TestConstraintSolverStrategy {
config.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, 0.0);
TopologyDetails topo = genTopology("testTopo", config, 2, 3, 30, 300, 0, 0, "user");
- Map<String, TopologyDetails> topoMap = new HashMap<String, TopologyDetails>();
+ Map<String, TopologyDetails> topoMap = new HashMap<>();
topoMap.put(topo.getId(), topo);
Topologies topologies = new Topologies(topoMap);
- Cluster cluster = new Cluster(new INimbusTest(), supMap, new HashMap<String, SchedulerAssignmentImpl>(), topologies, config);
+ Cluster cluster = new Cluster(new INimbusTest(), new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
rs.prepare(config);
rs.schedule(topologies, cluster);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
index 607b815..d552798 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestDefaultResourceAwareStrategy.java
@@ -62,6 +62,8 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.TreeSet;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
public class TestDefaultResourceAwareStrategy {
private static final Logger LOG = LoggerFactory.getLogger(TestDefaultResourceAwareStrategy.class);
@@ -128,7 +130,7 @@ public class TestDefaultResourceAwareStrategy {
genExecsAndComps(stormToplogy), CURRENT_TIME, "user");
Topologies topologies = new Topologies(topo);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, conf);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -195,7 +197,7 @@ public class TestDefaultResourceAwareStrategy {
genExecsAndComps(stormToplogy), CURRENT_TIME, "user");
Topologies topologies = new Topologies(topo);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, conf);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -264,7 +266,7 @@ public class TestDefaultResourceAwareStrategy {
TopologyDetails topo2 = genTopology("topo-2", config, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
Topologies topologies = new Topologies(topo1, topo2);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
List<String> supHostnames = new LinkedList<>();
for (SupervisorDetails sup : supMap.values()) {
@@ -388,7 +390,7 @@ public class TestDefaultResourceAwareStrategy {
TopologyDetails topo2 = genTopology("topo-2", t2Conf, 8, 0, 2, 0, CURRENT_TIME - 2, 10, "user");
Topologies topologies = new Topologies(topo1, topo2);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
List<String> supHostnames = new LinkedList<>();
for (SupervisorDetails sup : supMap.values()) {
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
index 2c41237..124247e 100644
--- a/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
+++ b/storm-server/src/test/java/org/apache/storm/scheduler/resource/strategies/scheduling/TestGenericResourceAwareStrategy.java
@@ -52,6 +52,9 @@ import org.slf4j.LoggerFactory;
import static org.apache.storm.scheduler.resource.TestUtilsForResourceAwareScheduler.*;
import static org.junit.Assert.*;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.scheduler.resource.normalization.ResourceMetrics;
+
public class TestGenericResourceAwareStrategy {
private static final Logger LOG = LoggerFactory.getLogger(TestGenericResourceAwareStrategy.class);
@@ -99,7 +102,7 @@ public class TestGenericResourceAwareStrategy {
Topologies topologies = new Topologies(topo);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, conf);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -187,7 +190,7 @@ public class TestGenericResourceAwareStrategy {
genExecsAndComps(stormToplogy), currentTime, "user");
Topologies topologies = new Topologies(topo);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, conf);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, conf);
ResourceAwareScheduler rs = new ResourceAwareScheduler();
@@ -241,7 +244,7 @@ public class TestGenericResourceAwareStrategy {
//Schedule the simple topology first
Topologies topologies = new Topologies(tdSimple);
- Cluster cluster = new Cluster(iNimbus, supMap, new HashMap<>(), topologies, config);
+ Cluster cluster = new Cluster(iNimbus, new ResourceMetrics(new StormMetricsRegistry()), supMap, new HashMap<>(), topologies, config);
rs.schedule(topologies, cluster);
TopologyBuilder builder = topologyBuilder(1, 5, 100, 300);
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
index 2815810..66653d1 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
public class DRPCServer implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(DRPCServer.class);
- private static final Meter meterShutdownCalls = StormMetricsRegistry.registerMeter("drpc:num-shutdown-calls");
+ private final Meter meterShutdownCalls;
//TODO in the future this might be better in a common webapp location
@@ -82,7 +82,7 @@ public class DRPCServer implements AutoCloseable {
ThriftConnectionType.DRPC_INVOCATIONS);
}
- private static Server mkHttpServer(Map<String, Object> conf, DRPC drpc) {
+ private static Server mkHttpServer(StormMetricsRegistry metricsRegistry, Map<String, Object> conf, DRPC drpc) {
Integer drpcHttpPort = (Integer) conf.get(DaemonConfig.DRPC_HTTP_PORT);
Server ret = null;
if (drpcHttpPort != null && drpcHttpPort >= 0) {
@@ -105,7 +105,7 @@ public class DRPCServer implements AutoCloseable {
final Boolean disableHttpBinding = (Boolean) (conf.get(DaemonConfig.DRPC_DISABLE_HTTP_BINDING));
//TODO a better way to do this would be great.
- DRPCApplication.setup(drpc);
+ DRPCApplication.setup(drpc, metricsRegistry);
ret = UIHelpers.jettyCreateServer(drpcHttpPort, null, httpsPort, disableHttpBinding);
UIHelpers.configSsl(ret, httpsPort, httpsKsPath, httpsKsPassword, httpsKsType, httpsKeyPassword,
@@ -135,13 +135,15 @@ public class DRPCServer implements AutoCloseable {
/**
* Constructor.
* @param conf Drpc conf for the servers
+ * @param metricsRegistry The metrics registry
*/
- public DRPCServer(Map<String, Object> conf) {
- drpc = new DRPC(conf);
+ public DRPCServer(Map<String, Object> conf, StormMetricsRegistry metricsRegistry) {
+ meterShutdownCalls = metricsRegistry.registerMeter("drpc:num-shutdown-calls");
+ drpc = new DRPC(metricsRegistry, conf);
DRPCThrift thrift = new DRPCThrift(drpc);
handlerServer = mkHandlerServer(thrift, ObjectReader.getInt(conf.get(Config.DRPC_PORT), null), conf);
invokeServer = mkInvokeServer(thrift, ObjectReader.getInt(conf.get(Config.DRPC_INVOCATIONS_PORT), 3773), conf);
- httpServer = mkHttpServer(conf, drpc);
+ httpServer = mkHttpServer(metricsRegistry, conf, drpc);
}
@VisibleForTesting
@@ -223,9 +225,13 @@ public class DRPCServer implements AutoCloseable {
public static void main(String [] args) throws Exception {
Utils.setupDefaultUncaughtExceptionHandler();
Map<String, Object> conf = ConfigUtils.readStormConfig();
- try (DRPCServer server = new DRPCServer(conf)) {
- Utils.addShutdownHookWithForceKillIn1Sec(server::close);
- StormMetricsRegistry.startMetricsReporters(conf);
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
+ try (DRPCServer server = new DRPCServer(conf, metricsRegistry)) {
+ metricsRegistry.startMetricsReporters(conf);
+ Utils.addShutdownHookWithForceKillIn1Sec(() -> {
+ metricsRegistry.stopMetricsReporters();
+ server.close();
+ });
server.start();
server.awaitTermination();
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
index 15cc4d7..f88bb00 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
@@ -26,10 +26,12 @@ import javax.ws.rs.core.Application;
import org.apache.storm.daemon.common.AuthorizationExceptionMapper;
import org.apache.storm.daemon.drpc.DRPC;
+import org.apache.storm.metric.StormMetricsRegistry;
@ApplicationPath("")
public class DRPCApplication extends Application {
private static DRPC _drpc;
+ private static StormMetricsRegistry metricsRegistry;
private final Set<Object> singletons = new HashSet<Object>();
/**
@@ -38,7 +40,7 @@ public class DRPCApplication extends Application {
* and adds them to a set which can be retrieved later.
*/
public DRPCApplication() {
- singletons.add(new DRPCResource(_drpc));
+ singletons.add(new DRPCResource(_drpc, metricsRegistry));
singletons.add(new DRPCExceptionMapper());
singletons.add(new AuthorizationExceptionMapper());
}
@@ -48,7 +50,8 @@ public class DRPCApplication extends Application {
return singletons;
}
- public static void setup(DRPC drpc) {
+ public static void setup(DRPC drpc, StormMetricsRegistry metricsRegistry) {
_drpc = drpc;
+ DRPCApplication.metricsRegistry = metricsRegistry;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
index 54efd19..dba12cf 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
@@ -33,18 +33,20 @@ import org.apache.storm.metric.StormMetricsRegistry;
@Path("/drpc/")
public class DRPCResource {
- private static final Meter meterHttpRequests = StormMetricsRegistry.registerMeter("drpc:num-execute-http-requests");
- private static final Timer responseDuration = StormMetricsRegistry.registerTimer("drpc:HTTP-request-response-duration");
+ private final Meter meterHttpRequests;
+ private final Timer responseDuration;
private final DRPC drpc;
- public DRPCResource(DRPC drpc) {
+ public DRPCResource(DRPC drpc, StormMetricsRegistry metricsRegistry) {
this.drpc = drpc;
+ this.meterHttpRequests = metricsRegistry.registerMeter("drpc:num-execute-http-requests");
+ this.responseDuration = metricsRegistry.registerTimer("drpc:HTTP-request-response-duration");
}
//TODO put in some better exception mapping...
//TODO move populateContext to a filter...
@POST
- @Path("/{func}")
+ @Path("/{func}")
public String post(@PathParam("func") String func, String args, @Context HttpServletRequest request) throws Exception {
meterHttpRequests.mark();
return responseDuration.time(() -> drpc.executeBlocking(func, args));
@@ -59,7 +61,7 @@ public class DRPCResource {
}
@GET
- @Path("/{func}")
+ @Path("/{func}")
public String get(@PathParam("func") String func, @Context HttpServletRequest request) throws Exception {
meterHttpRequests.mark();
return responseDuration.time(() -> drpc.executeBlocking(func, ""));
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/LogviewerServer.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/LogviewerServer.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/LogviewerServer.java
index c9ede6e..6eba28d 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/LogviewerServer.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/LogviewerServer.java
@@ -33,7 +33,7 @@ import java.util.Map;
import org.apache.storm.DaemonConfig;
import org.apache.storm.daemon.logviewer.utils.DirectoryCleaner;
-import org.apache.storm.daemon.logviewer.utils.ExceptionMeters;
+import org.apache.storm.daemon.logviewer.utils.ExceptionMeterNames;
import org.apache.storm.daemon.logviewer.utils.LogCleaner;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
import org.apache.storm.daemon.logviewer.webapp.LogviewerApplication;
@@ -57,11 +57,11 @@ import org.slf4j.LoggerFactory;
*/
public class LogviewerServer implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(LogviewerServer.class);
- private static final Meter meterShutdownCalls = StormMetricsRegistry.registerMeter("logviewer:num-shutdown-calls");
private static final String stormHome = System.getProperty(ConfigUtils.STORM_HOME);
public static final String STATIC_RESOURCE_DIRECTORY_PATH = stormHome + "/public";
+ private final Meter meterShutdownCalls;
- private static Server mkHttpServer(Map<String, Object> conf) {
+ private static Server mkHttpServer(StormMetricsRegistry metricsRegistry, Map<String, Object> conf) {
Integer logviewerHttpPort = (Integer) conf.get(DaemonConfig.LOGVIEWER_PORT);
Server ret = null;
if (logviewerHttpPort != null && logviewerHttpPort >= 0) {
@@ -85,7 +85,7 @@ public class LogviewerServer implements AutoCloseable {
final Boolean httpsNeedClientAuth = (Boolean) (conf.get(DaemonConfig.LOGVIEWER_HTTPS_NEED_CLIENT_AUTH));
final Boolean disableHttpBinding = (Boolean) (conf.get(DaemonConfig.LOGVIEWER_DISABLE_HTTP_BINDING));
- LogviewerApplication.setup(conf);
+ LogviewerApplication.setup(conf, metricsRegistry);
ret = UIHelpers.jettyCreateServer(logviewerHttpPort, null, httpsPort, disableHttpBinding);
UIHelpers.configSsl(ret, httpsPort, httpsKsPath, httpsKsPassword, httpsKsType, httpsKeyPassword,
@@ -121,16 +121,18 @@ public class LogviewerServer implements AutoCloseable {
/**
* Constructor.
* @param conf Logviewer conf for the servers
+ * @param metricsRegistry The metrics registry
*/
- public LogviewerServer(Map<String, Object> conf) {
- httpServer = mkHttpServer(conf);
+ public LogviewerServer(Map<String, Object> conf, StormMetricsRegistry metricsRegistry) {
+ httpServer = mkHttpServer(metricsRegistry, conf);
+ meterShutdownCalls = metricsRegistry.registerMeter("logviewer:num-shutdown-calls");
+ ExceptionMeterNames.registerMeters(metricsRegistry);
}
@VisibleForTesting
void start() throws Exception {
LOG.info("Starting Logviewer...");
if (httpServer != null) {
- StormMetricsRegistry.registerMetricSet(ExceptionMeters::getMetrics);
httpServer.start();
}
}
@@ -162,17 +164,22 @@ public class LogviewerServer implements AutoCloseable {
Utils.setupDefaultUncaughtExceptionHandler();
Map<String, Object> conf = ConfigUtils.readStormConfig();
+ StormMetricsRegistry metricsRegistry = new StormMetricsRegistry();
String logRoot = ConfigUtils.workerArtifactsRoot(conf);
File logRootDir = new File(logRoot);
logRootDir.mkdirs();
- WorkerLogs workerLogs = new WorkerLogs(conf, logRootDir);
- DirectoryCleaner directoryCleaner = new DirectoryCleaner();
-
- try (LogviewerServer server = new LogviewerServer(conf);
- LogCleaner logCleaner = new LogCleaner(conf, workerLogs, directoryCleaner, logRootDir)) {
- Utils.addShutdownHookWithForceKillIn1Sec(server::close);
+ WorkerLogs workerLogs = new WorkerLogs(conf, logRootDir, metricsRegistry);
+ DirectoryCleaner directoryCleaner = new DirectoryCleaner(metricsRegistry);
+
+ try (LogviewerServer server = new LogviewerServer(conf, metricsRegistry);
+ LogCleaner logCleaner = new LogCleaner(conf, workerLogs, directoryCleaner, logRootDir, metricsRegistry)) {
+ metricsRegistry.startMetricsReporters(conf);
+ Utils.addShutdownHookWithForceKillIn1Sec(() -> {
+ metricsRegistry.stopMetricsReporters();
+ server.close();
+ });
logCleaner.start();
- StormMetricsRegistry.startMetricsReporters(conf);
+
server.start();
server.awaitTermination();
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogDownloadHandler.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogDownloadHandler.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogDownloadHandler.java
index 7b2fbcb..bb5a3ae 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogDownloadHandler.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogDownloadHandler.java
@@ -24,6 +24,7 @@ import javax.ws.rs.core.Response;
import org.apache.storm.daemon.logviewer.utils.LogFileDownloader;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
+import org.apache.storm.metric.StormMetricsRegistry;
public class LogviewerLogDownloadHandler {
@@ -37,10 +38,12 @@ public class LogviewerLogDownloadHandler {
* @param daemonLogRoot root daemon log directory
* @param workerLogs {@link WorkerLogs}
* @param resourceAuthorizer {@link ResourceAuthorizer}
+ * @param metricsRegistry The logviewer metrics registry
*/
- public LogviewerLogDownloadHandler(String logRoot, String daemonLogRoot, WorkerLogs workerLogs, ResourceAuthorizer resourceAuthorizer) {
+ public LogviewerLogDownloadHandler(String logRoot, String daemonLogRoot, WorkerLogs workerLogs,
+ ResourceAuthorizer resourceAuthorizer, StormMetricsRegistry metricsRegistry) {
this.workerLogs = workerLogs;
- this.logFileDownloadHelper = new LogFileDownloader(logRoot, daemonLogRoot, resourceAuthorizer);
+ this.logFileDownloadHelper = new LogFileDownloader(logRoot, daemonLogRoot, resourceAuthorizer, metricsRegistry);
}
/**
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java
index 089d965..5f86eb3 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java
@@ -63,7 +63,7 @@ import javax.ws.rs.core.Response;
import org.apache.commons.lang.StringUtils;
import org.apache.storm.daemon.logviewer.LogviewerConstant;
import org.apache.storm.daemon.logviewer.utils.DirectoryCleaner;
-import org.apache.storm.daemon.logviewer.utils.ExceptionMeters;
+import org.apache.storm.daemon.logviewer.utils.ExceptionMeterNames;
import org.apache.storm.daemon.logviewer.utils.LogviewerResponseBuilder;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
@@ -77,11 +77,14 @@ import org.apache.storm.utils.ServerUtils;
import org.jooq.lambda.Unchecked;
public class LogviewerLogPageHandler {
- private static final Meter numPageRead = StormMetricsRegistry.registerMeter("logviewer:num-page-read");
+ private final Meter numPageRead;
+ private final Meter numFileOpenExceptions;
+ private final Meter numFileReadExceptions;
private final String logRoot;
private final String daemonLogRoot;
private final WorkerLogs workerLogs;
private final ResourceAuthorizer resourceAuthorizer;
+ private final DirectoryCleaner directoryCleaner;
/**
* Constructor.
@@ -90,14 +93,20 @@ public class LogviewerLogPageHandler {
* @param daemonLogRoot root daemon log directory
* @param workerLogs {@link WorkerLogs}
* @param resourceAuthorizer {@link ResourceAuthorizer}
+ * @param metricsRegistry The logviewer metrics registry
*/
public LogviewerLogPageHandler(String logRoot, String daemonLogRoot,
WorkerLogs workerLogs,
- ResourceAuthorizer resourceAuthorizer) {
+ ResourceAuthorizer resourceAuthorizer,
+ StormMetricsRegistry metricsRegistry) {
this.logRoot = logRoot;
this.daemonLogRoot = daemonLogRoot;
this.workerLogs = workerLogs;
this.resourceAuthorizer = resourceAuthorizer;
+ this.numPageRead = metricsRegistry.registerMeter("logviewer:num-page-read");
+ this.numFileOpenExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_OPEN_EXCEPTIONS);
+ this.numFileReadExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_READ_EXCEPTIONS);
+ this.directoryCleaner = new DirectoryCleaner(metricsRegistry);
}
/**
@@ -125,7 +134,7 @@ public class LogviewerLogPageHandler {
if (topoDirFiles != null) {
for (File portDir : topoDirFiles) {
if (portDir.getName().equals(port.toString())) {
- fileResults.addAll(DirectoryCleaner.getFilesForDir(portDir));
+ fileResults.addAll(directoryCleaner.getFilesForDir(portDir));
}
}
}
@@ -141,7 +150,7 @@ public class LogviewerLogPageHandler {
File[] topoDirFiles = topoDir.listFiles();
if (topoDirFiles != null) {
for (File portDir : topoDirFiles) {
- fileResults.addAll(DirectoryCleaner.getFilesForDir(portDir));
+ fileResults.addAll(directoryCleaner.getFilesForDir(portDir));
}
}
}
@@ -149,7 +158,7 @@ public class LogviewerLogPageHandler {
} else {
File portDir = ConfigUtils.getWorkerDirFromRoot(logRoot, topologyId, port);
if (portDir.exists()) {
- fileResults = DirectoryCleaner.getFilesForDir(portDir);
+ fileResults = directoryCleaner.getFilesForDir(portDir);
}
}
}
@@ -191,7 +200,7 @@ public class LogviewerLogPageHandler {
SortedSet<File> logFiles;
try {
logFiles = Arrays.stream(topoDir.listFiles())
- .flatMap(Unchecked.function(portDir -> DirectoryCleaner.getFilesForDir(portDir).stream()))
+ .flatMap(Unchecked.function(portDir -> directoryCleaner.getFilesForDir(portDir).stream()))
.filter(File::isFile)
.collect(toCollection(TreeSet::new));
} catch (UncheckedIOException e) {
@@ -325,10 +334,10 @@ public class LogviewerLogPageHandler {
try {
return isZipFile ? ServerUtils.zipFileSize(file) : file.length();
} catch (FileNotFoundException e) {
- ExceptionMeters.NUM_FILE_OPEN_EXCEPTIONS.mark();
+ numFileOpenExceptions.mark();
throw e;
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_READ_EXCEPTIONS.mark();
+ numFileReadExceptions.mark();
throw e;
}
}
@@ -460,10 +469,10 @@ public class LogviewerLogPageHandler {
numPageRead.mark();
return output.toString();
} catch (FileNotFoundException e) {
- ExceptionMeters.NUM_FILE_OPEN_EXCEPTIONS.mark();
+ numFileOpenExceptions.mark();
throw e;
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_READ_EXCEPTIONS.mark();
+ numFileReadExceptions.mark();
throw e;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandler.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandler.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandler.java
index bcde077..854b897 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandler.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogSearchHandler.java
@@ -25,7 +25,6 @@ import static org.apache.storm.daemon.utils.ListFunctionalSupport.last;
import static org.apache.storm.daemon.utils.ListFunctionalSupport.rest;
import static org.apache.storm.daemon.utils.PathUtil.truncatePathToLastElements;
-import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -61,7 +60,7 @@ import org.apache.storm.DaemonConfig;
import org.apache.storm.daemon.common.JsonResponseBuilder;
import org.apache.storm.daemon.logviewer.LogviewerConstant;
import org.apache.storm.daemon.logviewer.utils.DirectoryCleaner;
-import org.apache.storm.daemon.logviewer.utils.ExceptionMeters;
+import org.apache.storm.daemon.logviewer.utils.ExceptionMeterNames;
import org.apache.storm.daemon.logviewer.utils.LogviewerResponseBuilder;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
import org.apache.storm.daemon.logviewer.utils.WorkerLogs;
@@ -78,21 +77,24 @@ import org.slf4j.LoggerFactory;
public class LogviewerLogSearchHandler {
private static final Logger LOG = LoggerFactory.getLogger(LogviewerLogSearchHandler.class);
- private static final Meter numDeepSearchNoResult = StormMetricsRegistry.registerMeter("logviewer:num-deep-search-no-result");
- private static final Histogram numFileScanned = StormMetricsRegistry.registerHistogram("logviewer:num-files-scanned-per-deep-search");
- private static final Meter numSearchRequestNoResult = StormMetricsRegistry.registerMeter("logviewer:num-search-request-no-result");
-
public static final int GREP_MAX_SEARCH_SIZE = 1024;
public static final int GREP_BUF_SIZE = 2048;
public static final int GREP_CONTEXT_SIZE = 128;
public static final Pattern WORKER_LOG_FILENAME_PATTERN = Pattern.compile("^worker.log(.*)");
+ private final Meter numDeepSearchNoResult;
+ private final Histogram numFileScanned;
+ private final Meter numSearchRequestNoResult;
+ private final Meter numFileOpenExceptions;
+ private final Meter numFileReadExceptions;
+
private final Map<String, Object> stormConf;
private final String logRoot;
private final String daemonLogRoot;
private final ResourceAuthorizer resourceAuthorizer;
private final Integer logviewerPort;
private final String scheme;
+ private final DirectoryCleaner directoryCleaner;
/**
* Constructor.
@@ -101,9 +103,10 @@ public class LogviewerLogSearchHandler {
* @param logRoot log root directory
* @param daemonLogRoot daemon log root directory
* @param resourceAuthorizer {@link ResourceAuthorizer}
+ * @param metricsRegistry The logviewer metrics registry
*/
public LogviewerLogSearchHandler(Map<String, Object> stormConf, String logRoot, String daemonLogRoot,
- ResourceAuthorizer resourceAuthorizer) {
+ ResourceAuthorizer resourceAuthorizer, StormMetricsRegistry metricsRegistry) {
this.stormConf = stormConf;
this.logRoot = logRoot;
this.daemonLogRoot = daemonLogRoot;
@@ -116,6 +119,12 @@ public class LogviewerLogSearchHandler {
this.logviewerPort = ObjectReader.getInt(httpsPort);
this.scheme = "https";
}
+ this.numDeepSearchNoResult = metricsRegistry.registerMeter("logviewer:num-deep-search-no-result");
+ this.numFileScanned = metricsRegistry.registerHistogram("logviewer:num-files-scanned-per-deep-search");
+ this.numSearchRequestNoResult = metricsRegistry.registerMeter("logviewer:num-search-request-no-result");
+ this.numFileOpenExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_OPEN_EXCEPTIONS);
+ this.numFileReadExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_READ_EXCEPTIONS);
+ this.directoryCleaner = new DirectoryCleaner(metricsRegistry);
}
/**
@@ -382,11 +391,11 @@ public class LogviewerLogSearchHandler {
} catch (UnknownHostException | UnsupportedEncodingException e) {
throw new RuntimeException(e);
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_READ_EXCEPTIONS.mark();
+ numFileReadExceptions.mark();
throw new RuntimeException(e);
}
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_OPEN_EXCEPTIONS.mark();
+ numFileOpenExceptions.mark();
throw new RuntimeException(e);
}
}
@@ -402,7 +411,7 @@ public class LogviewerLogSearchHandler {
@VisibleForTesting
List<File> logsForPort(String user, File portDir) {
try {
- List<File> workerLogs = DirectoryCleaner.getFilesForDir(portDir).stream()
+ List<File> workerLogs = directoryCleaner.getFilesForDir(portDir).stream()
.filter(file -> WORKER_LOG_FILENAME_PATTERN.asPredicate().test(file.getName()))
.collect(toList());
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
index 202d421..d7e8ca6 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
@@ -28,6 +28,7 @@ import static j2html.TagCreator.title;
import static j2html.TagCreator.ul;
import static java.util.stream.Collectors.toList;
+import com.codahale.metrics.Meter;
import j2html.tags.DomContent;
import java.io.File;
@@ -37,25 +38,34 @@ import javax.ws.rs.core.Response;
import org.apache.commons.lang.StringUtils;
import org.apache.storm.daemon.logviewer.utils.DirectoryCleaner;
+import org.apache.storm.daemon.logviewer.utils.ExceptionMeterNames;
import org.apache.storm.daemon.logviewer.utils.LogviewerResponseBuilder;
import org.apache.storm.daemon.logviewer.utils.ResourceAuthorizer;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.ServerUtils;
public class LogviewerProfileHandler {
public static final String WORKER_LOG_FILENAME = "worker.log";
+
+ private final Meter numFileDownloadExceptions;
+
private final String logRoot;
private final ResourceAuthorizer resourceAuthorizer;
+ private final DirectoryCleaner directoryCleaner;
/**
* Constructor.
*
* @param logRoot worker log root directory
* @param resourceAuthorizer {@link ResourceAuthorizer}
+ * @param metricsRegistry The logviewer metrisc registry
*/
- public LogviewerProfileHandler(String logRoot, ResourceAuthorizer resourceAuthorizer) {
+ public LogviewerProfileHandler(String logRoot, ResourceAuthorizer resourceAuthorizer, StormMetricsRegistry metricsRegistry) {
this.logRoot = logRoot;
this.resourceAuthorizer = resourceAuthorizer;
+ this.numFileDownloadExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_DOWNLOAD_EXCEPTIONS);
+ this.directoryCleaner = new DirectoryCleaner(metricsRegistry);
}
/**
@@ -101,7 +111,7 @@ public class LogviewerProfileHandler {
if (dir.exists() && file.exists()) {
String workerFileRelativePath = String.join(File.separator, topologyId, portStr, WORKER_LOG_FILENAME);
if (resourceAuthorizer.isUserAllowedToAccessFile(user, workerFileRelativePath)) {
- return LogviewerResponseBuilder.buildDownloadFile(file);
+ return LogviewerResponseBuilder.buildDownloadFile(file, numFileDownloadExceptions);
} else {
return LogviewerResponseBuilder.buildResponseUnauthorizedUser(user);
}
@@ -129,7 +139,7 @@ public class LogviewerProfileHandler {
}
private List<String> getProfilerDumpFiles(File dir) throws IOException {
- List<File> filesForDir = DirectoryCleaner.getFilesForDir(dir);
+ List<File> filesForDir = directoryCleaner.getFilesForDir(dir);
return filesForDir.stream().filter(file -> {
String fileName = file.getName();
return StringUtils.isNotEmpty(fileName)
http://git-wip-us.apache.org/repos/asf/storm/blob/8c90f12d/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/DirectoryCleaner.java
----------------------------------------------------------------------
diff --git a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/DirectoryCleaner.java b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/DirectoryCleaner.java
index 293b2be..0b17c84 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/DirectoryCleaner.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/DirectoryCleaner.java
@@ -18,6 +18,7 @@
package org.apache.storm.daemon.logviewer.utils;
+import com.codahale.metrics.Meter;
import java.io.File;
import java.io.IOException;
import java.nio.file.DirectoryStream;
@@ -30,6 +31,7 @@ import java.util.PriorityQueue;
import java.util.Set;
import java.util.Stack;
import java.util.regex.Pattern;
+import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
@@ -51,7 +53,11 @@ public class DirectoryCleaner {
private static final int MAX_ROUNDS = 512; // max rounds of scanning the dirs
public static final int MAX_NUMBER_OF_FILES_FOR_DIR = 1024;
- // not defining this as static is to allow for mocking in tests
+ private final Meter numFileOpenExceptions;
+
+ public DirectoryCleaner(StormMetricsRegistry metricsRegistry) {
+ this.numFileOpenExceptions = metricsRegistry.registerMeter(ExceptionMeterNames.NUM_FILE_OPEN_EXCEPTIONS);
+ }
/**
* Creates DirectoryStream for give directory.
@@ -63,7 +69,7 @@ public class DirectoryCleaner {
try {
return Files.newDirectoryStream(dir.toPath());
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_OPEN_EXCEPTIONS.mark();
+ numFileOpenExceptions.mark();
throw e;
}
}
@@ -182,8 +188,8 @@ public class DirectoryCleaner {
* @param dir directory to get file list
* @return files in directory
*/
- public static List<File> getFilesForDir(File dir) throws IOException {
- List<File> files = new ArrayList<File>();
+ public List<File> getFilesForDir(File dir) throws IOException {
+ List<File> files = new ArrayList<>();
try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir.toPath())) {
for (Path path : stream) {
files.add(path.toFile());
@@ -192,7 +198,7 @@ public class DirectoryCleaner {
}
}
} catch (IOException e) {
- ExceptionMeters.NUM_FILE_OPEN_EXCEPTIONS.mark();
+ numFileOpenExceptions.mark();
throw e;
}
return files;