|
@@ -31,6 +31,7 @@ import java.util.concurrent.ScheduledFuture;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.function.Function;
|
|
import java.util.function.Function;
|
|
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@@ -57,29 +58,30 @@ import static org.apache.hadoop.metrics2.lib.Interns.*;
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Public
|
|
@InterfaceAudience.Public
|
|
@InterfaceStability.Evolving
|
|
@InterfaceStability.Evolving
|
|
-public class RollingAverages extends MutableMetric implements Closeable {
|
|
|
|
|
|
+public class MutableRollingAverages extends MutableMetric implements Closeable {
|
|
|
|
|
|
- private final MutableRatesWithAggregation innerMetrics =
|
|
|
|
|
|
+ private MutableRatesWithAggregation innerMetrics =
|
|
new MutableRatesWithAggregation();
|
|
new MutableRatesWithAggregation();
|
|
|
|
|
|
- private static final ScheduledExecutorService SCHEDULER = Executors
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ static final ScheduledExecutorService SCHEDULER = Executors
|
|
.newScheduledThreadPool(1, new ThreadFactoryBuilder().setDaemon(true)
|
|
.newScheduledThreadPool(1, new ThreadFactoryBuilder().setDaemon(true)
|
|
- .setNameFormat("RollingAverages-%d").build());
|
|
|
|
|
|
+ .setNameFormat("MutableRollingAverages-%d").build());
|
|
|
|
|
|
private ScheduledFuture<?> scheduledTask = null;
|
|
private ScheduledFuture<?> scheduledTask = null;
|
|
|
|
|
|
@Nullable
|
|
@Nullable
|
|
private Map<String, MutableRate> currentSnapshot;
|
|
private Map<String, MutableRate> currentSnapshot;
|
|
|
|
|
|
- private final int numWindows;
|
|
|
|
private final String avgInfoNameTemplate;
|
|
private final String avgInfoNameTemplate;
|
|
private final String avgInfoDescTemplate;
|
|
private final String avgInfoDescTemplate;
|
|
|
|
+ private int numWindows;
|
|
|
|
|
|
private static class SumAndCount {
|
|
private static class SumAndCount {
|
|
private final double sum;
|
|
private final double sum;
|
|
private final long count;
|
|
private final long count;
|
|
|
|
|
|
- public SumAndCount(final double sum, final long count) {
|
|
|
|
|
|
+ SumAndCount(final double sum, final long count) {
|
|
this.sum = sum;
|
|
this.sum = sum;
|
|
this.count = count;
|
|
this.count = count;
|
|
}
|
|
}
|
|
@@ -105,44 +107,36 @@ public class RollingAverages extends MutableMetric implements Closeable {
|
|
private Map<String, LinkedBlockingDeque<SumAndCount>> averages =
|
|
private Map<String, LinkedBlockingDeque<SumAndCount>> averages =
|
|
new ConcurrentHashMap<>();
|
|
new ConcurrentHashMap<>();
|
|
|
|
|
|
|
|
+ private static final long WINDOW_SIZE_MS_DEFAULT = 300_000;
|
|
|
|
+ private static final int NUM_WINDOWS_DEFAULT = 36;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
- * Constructor of {@link RollingAverages}.
|
|
|
|
- * @param windowSizeMs
|
|
|
|
- * The number of milliseconds of each window for which subset
|
|
|
|
- * of samples are gathered to compute the rolling average, A.K.A.
|
|
|
|
- * roll over interval.
|
|
|
|
- * @param numWindows
|
|
|
|
- * The number of windows maintained to compute the rolling average.
|
|
|
|
- * @param valueName
|
|
|
|
- * of the metric (e.g. "Time", "Latency")
|
|
|
|
|
|
+ * Constructor for {@link MutableRollingAverages}.
|
|
|
|
+ * @param metricValueName
|
|
*/
|
|
*/
|
|
- public RollingAverages(
|
|
|
|
- final long windowSizeMs,
|
|
|
|
- final int numWindows,
|
|
|
|
- final String valueName) {
|
|
|
|
- String uvName = StringUtils.capitalize(valueName);
|
|
|
|
- String lvName = StringUtils.uncapitalize(valueName);
|
|
|
|
- avgInfoNameTemplate = "[%s]" + "RollingAvg"+ uvName;
|
|
|
|
- avgInfoDescTemplate = "Rolling average "+ lvName +" for "+ "%s";
|
|
|
|
- this.numWindows = numWindows;
|
|
|
|
|
|
+ public MutableRollingAverages(String metricValueName) {
|
|
|
|
+ if (metricValueName == null) {
|
|
|
|
+ metricValueName = "";
|
|
|
|
+ }
|
|
|
|
+ avgInfoNameTemplate = "[%s]" + "RollingAvg" +
|
|
|
|
+ StringUtils.capitalize(metricValueName);
|
|
|
|
+ avgInfoDescTemplate = "Rolling average " +
|
|
|
|
+ StringUtils.uncapitalize(metricValueName) +" for "+ "%s";
|
|
|
|
+ numWindows = NUM_WINDOWS_DEFAULT;
|
|
scheduledTask = SCHEDULER.scheduleAtFixedRate(new RatesRoller(this),
|
|
scheduledTask = SCHEDULER.scheduleAtFixedRate(new RatesRoller(this),
|
|
- windowSizeMs, windowSizeMs, TimeUnit.MILLISECONDS);
|
|
|
|
|
|
+ WINDOW_SIZE_MS_DEFAULT, WINDOW_SIZE_MS_DEFAULT, TimeUnit.MILLISECONDS);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Constructor of {@link RollingAverages}.
|
|
|
|
- * @param windowSizeMs
|
|
|
|
- * The number of seconds of each window for which sub set of samples
|
|
|
|
- * are gathered to compute rolling average, also A.K.A roll over
|
|
|
|
- * interval.
|
|
|
|
- * @param numWindows
|
|
|
|
- * The number of windows maintained in the same time to compute the
|
|
|
|
- * average of the rolling averages.
|
|
|
|
|
|
+ * This method is for testing only to replace the scheduledTask.
|
|
*/
|
|
*/
|
|
- public RollingAverages(
|
|
|
|
- final long windowSizeMs,
|
|
|
|
- final int numWindows) {
|
|
|
|
- this(windowSizeMs, numWindows, "Time");
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ synchronized void replaceScheduledTask(int windows, long interval,
|
|
|
|
+ TimeUnit timeUnit) {
|
|
|
|
+ numWindows = windows;
|
|
|
|
+ scheduledTask.cancel(true);
|
|
|
|
+ scheduledTask = SCHEDULER.scheduleAtFixedRate(new RatesRoller(this),
|
|
|
|
+ interval, interval, timeUnit);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -190,9 +184,9 @@ public class RollingAverages extends MutableMetric implements Closeable {
|
|
}
|
|
}
|
|
|
|
|
|
private static class RatesRoller implements Runnable {
|
|
private static class RatesRoller implements Runnable {
|
|
- private final RollingAverages parent;
|
|
|
|
|
|
+ private final MutableRollingAverages parent;
|
|
|
|
|
|
- public RatesRoller(final RollingAverages parent) {
|
|
|
|
|
|
+ RatesRoller(final MutableRollingAverages parent) {
|
|
this.parent = parent;
|
|
this.parent = parent;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -218,7 +212,7 @@ public class RollingAverages extends MutableMetric implements Closeable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Iterates over snapshot to capture all Avg metrics into rolling structure
|
|
* Iterates over snapshot to capture all Avg metrics into rolling structure
|
|
- * {@link RollingAverages#averages}.
|
|
|
|
|
|
+ * {@link MutableRollingAverages#averages}.
|
|
*/
|
|
*/
|
|
private synchronized void rollOverAvgs() {
|
|
private synchronized void rollOverAvgs() {
|
|
if (currentSnapshot == null) {
|
|
if (currentSnapshot == null) {
|
|
@@ -232,7 +226,7 @@ public class RollingAverages extends MutableMetric implements Closeable {
|
|
new Function<String, LinkedBlockingDeque<SumAndCount>>() {
|
|
new Function<String, LinkedBlockingDeque<SumAndCount>>() {
|
|
@Override
|
|
@Override
|
|
public LinkedBlockingDeque<SumAndCount> apply(String k) {
|
|
public LinkedBlockingDeque<SumAndCount> apply(String k) {
|
|
- return new LinkedBlockingDeque<SumAndCount>(numWindows);
|
|
|
|
|
|
+ return new LinkedBlockingDeque<>(numWindows);
|
|
}
|
|
}
|
|
});
|
|
});
|
|
final SumAndCount sumAndCount = new SumAndCount(
|
|
final SumAndCount sumAndCount = new SumAndCount(
|