Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

STORM-3133: Extend metrics on Nimbus and LogViewer #2754

Merged
merged 1 commit into from Aug 10, 2018

Conversation

zd-project
Copy link
Contributor

@zd-project zd-project commented Jul 5, 2018

This PR is based on #2743, #2771 , #2787 , and #2788 and implements the following metrics:

LogViewer:

  • Time to complete one clean up loop Time. (timing on blocking calls)
  • Search request Cnt: By category - search (non-archived only) vs deep search (archived/non-archived)
  • Search Request - Response time - search vs deep search (both timing on blocking calls)
  • Search Request - 0 result Cnt - search vs deep search
    • Search Result - open files (scanned files, deep search only, manually tested)
  • File Download request Cnt/ And Size served (round to MB)
  • Failures/exceptions.
    • Does not track common IO exceptions thrown by File#getCanonicalFile/File#getCanonicalPath(), as it's common but does not actually try to open or read file content
    • Does not track meta file IO exception
  • File partial read count
  • Disk usage by worker logs
  • Disk free-up from each LogCleaner routine

Nimbus:

  • File upload time.
  • Nimbus launch/relaunch count
  • Nimbus loss of leadership
    • meter marking when a nimbus node gains or loses leadership
  • Scheduling latency (both distribution (as histogram) and maximum (as gauge, including current scheduling iteration))
  • Scheduling throughput

@zd-project
Copy link
Contributor Author

zd-project commented Jul 5, 2018

One thing that is worthy of noticing: Storm UI currently interweaves different search API regarding searching functionalities and it's kind of confusing.

Specifically:

  1. For the search button at homepage, it uses a single deep search API to search all ports (server side process), both archived and non-archived.
  2. For non-archived search at a specific topology page, it invokes search API on each port inside a loop (client side process).
  3. For archived search at a specific topology page, it invokes deep search API (search-archived=on) on each port inside a loop (client side process)

As a result, metrics for these APIs may not accurately reflect how many searches are invoked from client's perspective.

@zd-project zd-project force-pushed the STORM-3133 branch 2 times, most recently from 6085270 to 07ad8a6 Compare July 12, 2018 16:16
@zd-project zd-project changed the title STORM-3133: Extend metrics on Nimbus and LogViewer [WIP] STORM-3133: Extend metrics on Nimbus and LogViewer Jul 19, 2018
@zd-project zd-project force-pushed the STORM-3133 branch 7 times, most recently from b5487c3 to c9a7937 Compare August 3, 2018 20:12
@zd-project zd-project changed the title [WIP] STORM-3133: Extend metrics on Nimbus and LogViewer STORM-3133: Extend metrics on Nimbus and LogViewer Aug 3, 2018
@zd-project
Copy link
Contributor Author

The metrics of IO exceptions on logviewer is kind of fuzzy as I go through the call stack manually and eyeballed all thrown exceptions. I'm not sure if it's complete or if it will over-count. It'd be great if you can help review it especially.

@zd-project zd-project force-pushed the STORM-3133 branch 3 times, most recently from 63fac98 to a1b3e02 Compare August 8, 2018 15:20
Copy link
Contributor

@Ethanlm Ethanlm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code changes look very good. Left some comments. Mostly nits

@@ -51,6 +51,7 @@ public void run() {
private SupervisorWorkerHeartbeats getAndResetWorkerHeartbeats() {
Map<String, LSWorkerHeartbeat> localHeartbeats;
try {
//TODO: This call no longer throws exceptions, do we still want to wrap it in try catch block?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove comment

@@ -48,6 +53,30 @@ public static Meter registerMeter(String name) {
return REGISTRY.register(name, new Meter());
}

//Change the name to avoid name conflict in future Metrics release
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change the name?

//This is the same as what #pageFile(String path, Integer tail) does
// boolean isZipFile = path.endsWith(".gz");
// long fileLength = isZipFile ? ServerUtils.zipFileSize(new File(path)) : new File(path).length();
// return pageFile(path, Long.valueOf(fileLength - tail).intValue(), tail);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove comments

} else {
logString = escapeHtml("This is a binary file and cannot display! You may download the full file.");
final boolean isZipFile = path.endsWith(".gz");
long fileLength = isZipFile ? ServerUtils.zipFileSize(file) : file.length();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not reuse getFileLength

@@ -71,6 +78,11 @@

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 numFilesOpenedDeepSearch = StormMetricsRegistry.registerHistogram(
"logviewer:num-files-opened-deep-search", new ExponentiallyDecayingReservoir());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

opened--> scanned? more clear

}
}
LOG.info("{} assignments unchanged: {}", difference.entriesInCommon().size(), difference.entriesInCommon().keySet());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to put to LOG.debug().

numAddedSlotPerScheduling.update(numAddedSlot);
numRemovedExecPerScheduling.update(numRemovedExec);
numRemovedSlotPerScheduling.update(numRemovedSlot);
numNetExecChangePerScheduling.update(numAddedExec - numRemovedExec);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to use numNetAddedExecPerScheduling and numNetAddedSlotPerScheduling

long scheduleTimeElapsedMs = System.currentTimeMillis() - beforeSchedule;
LOG.debug("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
scheduleTopologyTimeMs.update(scheduleTimeElapsedMs);
//Will compiler optimize the order of evalutation and cause race condition?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

idToResources.set(new HashMap<>());
idToWorkerResources.set(new HashMap<>());
}

//tasks figure out what tasks to talk to by looking at topology at runtime
// only log/set when there's been a change to the assignment
// TODO: why do we have loop fission here
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove comment

//fileOffset is not being used and it behaves inconsistently (showing
// (index of files search ends on - 1) if [enough matches] else (index of files search ends on))
// I don't think we should expose the data to public if it's not used.
// can I dropped this field or change its behavior so it's used for metrics [numScannedFiles]?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can file a separate jira for this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zd-project
Copy link
Contributor Author

Some important changes have been made to address nits you guys have, especially on metrics for scheduler. @srdo @Ethanlm

private static final Histogram numNetExecChangePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-executors-changed-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numNetSlotChangePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-slots-changed-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numNetExecIncreasePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-executors-changed-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numNetSlotIncreasePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-slots-changed-per-scheduling", new ExponentiallyDecayingReservoir());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to change nimbus:num-net-slots-changed-per-scheduling too

() -> {
try {
if (isLeader()) {
sendClusterMetricsToExecutors();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like a typo here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What typo?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry a mistake

Copy link
Contributor

@srdo srdo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Finished reviewing, left some comments.

void registerMetricSet() {
Meter existingInnerMeter = StormMetricsRegistry.registerMeter(name(INNER_SET, INNER_METER));

System.out.println("register outer set");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please use a Logger instead, I think System.out ends up in the console, whereas all the other logs go to files when we run tests.

@@ -52,7 +55,7 @@
/**
* The main entry of Logviewer.
*/
public class LogviewerServer implements AutoCloseable {
public class LogviewerServer implements AutoCloseable, MetricSet {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems like an odd choice. Why doesn't ExceptionMeters implement MetricSet instead, LogviewerServer is not a MetricSet?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be weirder since getMetrics is an instance method. So I have to do [SOME_ENUM].getMetrics()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You're right, but can't a lambda solve this? Would it work to do registerMetricSet(ExceptionMeters::getMetrics) in line 132 instead?

Copy link
Contributor Author

@zd-project zd-project Aug 9, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No it would be registerMetricSet(ExceptionMeters.[SOME_ENUM]) or

enum ExceptionMeters implements MetricSet {
    ENUM1;

    private static final Map<...> map = ...;

    public static MetricSet getMetricSet() {
        return ENUM1;

    @Override
    public Map<...> getMetrics() {
        return map;
    }
}

How pretty is that.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can think of an Alternative

enum ExceptionMeters {
    ENUM1;

    private static final Map<...> map = ...;
    private static final MetricSet set = () -> map;

    public static MetricSet getMetricSet() {
        return set;
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

StormMetricsRegistry.registerMetricSet(ExceptionMeters::getMetrics); works fine for me?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to be clear, I'm talking about removing implements MetricSet from LogviewerServer and replacing it with StormMetricsRegistry.registerMetricSet(ExceptionMeters::getMetrics);. ExceptionMeters doesn't need to implement MetricSet.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah I see. That's brilliant. Thanks


List<String> reorderedFilesStr = new ArrayList<>();
reorderedFilesStr.addAll(filesStrWithoutFileParam);
List<String> reorderedFilesStr = logFiles.stream().map(WorkerLogs::getTopologyPortWorkerLog)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Put each step of the pipeline on a new line.

@@ -71,6 +78,11 @@

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 numFilesOpenedDeepSearch = StormMetricsRegistry.registerHistogram(
"logviewer:num-files-opened-deep-search", new ExponentiallyDecayingReservoir());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you added a convenience histogram method to the registry at one point, so you don't have to specify the reservoir.

currentFileMatch.put("fileName", fileName);
Path firstLogAbsPath;
try {
firstLogAbsPath = firstLog.getCanonicalFile().toPath();
} catch (IOException e) {
throw new RuntimeException(e);
}
//Why do we need to start from scratch to retrieve just the port here?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove. Also what do you mean start from scratch?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh I was complaining that the implementation doesn't look too good. Both port and filename are included in the absolute path. We could just optimize the code here. It looks minor though.

private static final Histogram numAddedSlotPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-added-slots-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numRemovedExecPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-removed-executors-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numRemovedSlotPerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-removed-slots-per-scheduling", new ExponentiallyDecayingReservoir());
private static final Histogram numNetExecIncreasePerScheduling = StormMetricsRegistry.registerHistogram("nimbus:num-net-executors-changed-per-scheduling", new ExponentiallyDecayingReservoir());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The variable name says it's exec increase, but the metric name says it's the number changed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry I fixed in in another commit. Will port it here.

import org.apache.storm.utils.ConfigUtils;
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");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is this metric useful for?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How many pages of logs are read from handling each request.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean why does someone want to know this :P. Nevermind, I'm guessing this was also requested by @kishorvpatil. I just wanted to know whether someone asked for/had a use case for these metrics.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's as part of IO performance metrics. He seems to be concerning about the IO performance of logviewer, which makes sense to me because majority of our API calls are blocking call.

@@ -71,6 +78,10 @@

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");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are these metrics useful?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's for analyzing invalid requests, but I'm not sure. @kishorvpatil requested it. You may ask him for his intention.


public enum ExceptionMeters {
//Operation level IO Exceptions
NUM_FILE_OPEN_EXCEPTIONS("logviewer:num-file-open-exceptions"),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is it useful to distinguish between e.g. read and open errors in the metrics?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good point though. I don't see a very compelling reason. Again this was requested by @kishorvpatil.

@@ -62,6 +64,14 @@
"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");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see why it's useful to know how long they take, but is it useful to know how many searches have happened as well?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It also reports the frequency of requests handling. I think it can reflect the server load in a way.

Copy link
Contributor

@Ethanlm Ethanlm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 looks good to me. I think it's good to go after you address @srdo 's comments.

@zd-project
Copy link
Contributor Author

All nits should have been addressed as of now.

long scheduleTimeElapsedMs = System.currentTimeMillis() - beforeSchedule;
LOG.debug("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
scheduleTopologyTimeMs.update(scheduleTimeElapsedMs);
long elapsed = -schedulingStartTimeNs.getAndSet(null) + Time.nanoTime();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I might have asked this over on the other PR, but is it written this way to fix the race? If so, please put a note in here about it, otherwise it's very likely this gets reordered next time someone happens across this code.

Also splitting this line in two would probably also be appropriate.

//Get and set the start time before getting current time in order to avoid potential race with the longestSchedulingTime gauge
long startTimeNs = schedulingStartTimeNs.getAndSet(null);
long nowNs = Time.nanoTime();
long elapsed = nowNs - startTimeNs;

It's much more obvious that the order matters this way.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay.


StormMetricsRegistry.registerGauge("nimbus:longest-scheduling-time-ms", () -> {
//We want to update longest scheduling time in real time in case scheduler get stuck
// It's normal to see some very minor jiggling in value as race condition may happen
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the race still there given your reordering?

@srdo
Copy link
Contributor

srdo commented Aug 10, 2018

+1. Please squash and we can merge.

STORM-3157: Added registration method for MetricSet

STORM-3133: Refactored and added metrics to LogViewer components

STORM-3133: Fixed up Unit test for LogViewer

STORM-3133: Refactored and added metrics to Nimbus components.

STORM-3133: Add nimbus scheduling metrics

 STORM-3133: Add metrics for disk usage of workers' logs and performance of LogCleaner routine

STORM-3133: Refactored code and added file partial read count metric for logviewer

STORM-3133: Add metrics for counting LogViewer's IOExceptions
@asfgit asfgit merged commit bf81b68 into apache:master Aug 10, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants