Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -559,18 +559,19 @@ private void fixAnomalyInProgress(AnomalyType anomalyType) throws Exception {
skipReportingIfNotUpdated = anomalyType == KafkaAnomalyType.BROKER_FAILURE;
throw ofe;
} finally {
handlePostFixAnomaly(isReadyToFix, fixStarted, anomalyId, skipReportingIfNotUpdated);
handlePostFixAnomaly(isReadyToFix, fixStarted, anomalyId, skipReportingIfNotUpdated, anomalyType);
}
}
}
}

private void handlePostFixAnomaly(boolean isReadyToFix, boolean fixStarted, String anomalyId, boolean skipReportingIfNotUpdated) {
private void handlePostFixAnomaly(boolean isReadyToFix, boolean fixStarted, String anomalyId, boolean skipReportingIfNotUpdated,
AnomalyType anomalyType) {
if (isReadyToFix) {
_anomalyDetectorState.onAnomalyHandle(_anomalyInProgress, fixStarted ? AnomalyState.Status.FIX_STARTED
: AnomalyState.Status.FIX_FAILED_TO_START);
if (fixStarted) {
_anomalyDetectorState.incrementNumSelfHealingStarted();
_anomalyDetectorState.incrementNumSelfHealingStarted(anomalyType);
LOG.info("[{}] Self-healing started successfully.", anomalyId);
} else {
_anomalyDetectorState.incrementNumSelfHealingFailedToStart();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@

package com.linkedin.kafka.cruisecontrol.detector;

import com.codahale.metrics.Counter;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricRegistry;
Expand Down Expand Up @@ -81,6 +82,8 @@ public class AnomalyDetectorState {
private double _balancednessScore;
private boolean _hasUnfixableGoals;
private final Map<AnomalyType, Timer> _anomalyDetectToFixCompleteTimer;
private final Map<AnomalyType, Counter> _numSelfHealingStartedPerAnomaly;
private final Map<AnomalyType, Counter> _numSelfHealingEndedPerAnomaly;

public AnomalyDetectorState(Time time,
AnomalyNotifier anomalyNotifier,
Expand Down Expand Up @@ -125,6 +128,8 @@ protected boolean removeEldestEntry(Map.Entry<String, AnomalyState> eldest) {
(Gauge<Integer>) () -> hasUnfixableGoals() ? 1 : 0);

_anomalyRateByType = new HashMap<>();
_numSelfHealingStartedPerAnomaly = new HashMap<>();
_numSelfHealingEndedPerAnomaly = new HashMap<>();
_anomalyRateByType.put(BROKER_FAILURE,
dropwizardMetricRegistry.meter(MetricRegistry.name(ANOMALY_DETECTOR_SENSOR, "broker-failure-rate")));
_anomalyRateByType.put(GOAL_VIOLATION,
Expand All @@ -138,12 +143,24 @@ protected boolean removeEldestEntry(Map.Entry<String, AnomalyState> eldest) {
_anomalyRateByType.put(MAINTENANCE_EVENT,
dropwizardMetricRegistry.meter(MetricRegistry.name(ANOMALY_DETECTOR_SENSOR, "maintenance-event-rate")));
for (KafkaAnomalyType anomalyType : KafkaAnomalyType.cachedValues()) {
Timer timer = dropwizardMetricRegistry.timer(
MetricRegistry.name(ANOMALY_DETECTOR_SENSOR, String.format("%s-detect-to-fix-complete-timer", anomalyType.toString().toLowerCase())));
_anomalyDetectToFixCompleteTimer.put(anomalyType, timer);
_anomalyDetectToFixCompleteTimer.put(anomalyType,
dropwizardMetricRegistry.timer(MetricRegistry.name(ANOMALY_DETECTOR_SENSOR,
String.format("%s-detect-to-fix-complete-timer", anomalyType.toString().toLowerCase()))));

_numSelfHealingStartedPerAnomaly.put(anomalyType,
dropwizardMetricRegistry.counter(MetricRegistry.name(ANOMALY_DETECTOR_SENSOR,
String.format("num-of-self-healing-started-for-%s", anomalyType.toString().toLowerCase()))));

_numSelfHealingEndedPerAnomaly.put(anomalyType,
dropwizardMetricRegistry.counter(MetricRegistry.name(ANOMALY_DETECTOR_SENSOR,
String.format("num-of-self-healing-ended-for-%s", anomalyType.toString().toLowerCase()))));
}
} else {
_anomalyRateByType = new HashMap<>();
_numSelfHealingStartedPerAnomaly = new HashMap<>();
_numSelfHealingEndedPerAnomaly = new HashMap<>();
KafkaAnomalyType.cachedValues().forEach(anomalyType -> _numSelfHealingStartedPerAnomaly.put(anomalyType, new Counter()));
KafkaAnomalyType.cachedValues().forEach(anomalyType -> _numSelfHealingEndedPerAnomaly.put(anomalyType, new Counter()));
KafkaAnomalyType.cachedValues().forEach(anomalyType -> _anomalyRateByType.put(anomalyType, new Meter()));
}
_balancednessScore = INITIAL_BALANCEDNESS_SCORE;
Expand Down Expand Up @@ -259,9 +276,21 @@ long numSelfHealingStarted() {

/**
* Increment the number of self healing actions started successfully.
*
* @param anomalyType Type of anomaly
*/
void incrementNumSelfHealingStarted() {
void incrementNumSelfHealingStarted(AnomalyType anomalyType) {
_numSelfHealingStarted.incrementAndGet();
_numSelfHealingStartedPerAnomaly.get(anomalyType).inc();
}

/**
* Increment the number of self healing actions ended successfully per anomaly.
*
* @param anomalyType Type of anomaly
*/
void incrementNumSelfHealingEndedPerAnomaly(AnomalyType anomalyType) {
_numSelfHealingEndedPerAnomaly.get(anomalyType).inc();
}

/**
Expand Down Expand Up @@ -309,6 +338,7 @@ public synchronized void markSelfHealingFinished(String anomalyId, boolean compl
// Time the duration if the self-healing is completed successfully.
// completeWithError is true if the proposal execution is interrupted (receive stop signal) or encountered exception.
// execution-stopped metrics track the number of stopped execution.
incrementNumSelfHealingEndedPerAnomaly(_ongoingSelfHealingAnomaly.anomalyType());
_anomalyDetectToFixCompleteTimer.get(_ongoingSelfHealingAnomaly.anomalyType()).update(totalTime, TimeUnit.MILLISECONDS);
}
_ongoingSelfHealingAnomaly = null;
Expand Down
Loading