mirror of https://github.com/apache/kafka.git
KAFKA-2668; Add a metric that records the total number of metrics
onurkaraman becketqin Do you have time to review this patch? It addresses the ticket that jjkoshy filed in KAFKA-2668. Author: Dong Lin <lindong28@gmail.com> Reviewers: Onur Karaman <okaraman@linkedin.com>, Joel Koshy <jjkoshy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>, Jun Rao <junrao@gmail.com> Closes #328 from lindong28/KAFKA-2668
This commit is contained in:
parent
ee6b5e044c
commit
ef92a8ae74
|
|
@ -531,12 +531,14 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
throw new ConfigException(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG + " should be greater than " + ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG + " and " + ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
|
||||
this.time = new SystemTime();
|
||||
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
|
||||
TimeUnit.MILLISECONDS);
|
||||
clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
|
||||
if (clientId.length() <= 0)
|
||||
clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
|
||||
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
|
||||
metricsTags.put("client-id", clientId);
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS)
|
||||
.tags(metricsTags);
|
||||
List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
|
||||
MetricsReporter.class);
|
||||
reporters.add(new JmxReporter(JMX_PREFIX));
|
||||
|
|
@ -546,11 +548,9 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
this.metadata.update(Cluster.bootstrap(addresses), 0);
|
||||
String metricGrpPrefix = "consumer";
|
||||
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
|
||||
metricsTags.put("client-id", clientId);
|
||||
ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values());
|
||||
NetworkClient netClient = new NetworkClient(
|
||||
new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags, channelBuilder),
|
||||
new Selector(config.getLong(ConsumerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder),
|
||||
this.metadata,
|
||||
clientId,
|
||||
100, // a fixed large enough value will suffice
|
||||
|
|
@ -573,7 +573,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
this.subscriptions,
|
||||
metrics,
|
||||
metricGrpPrefix,
|
||||
metricsTags,
|
||||
this.time,
|
||||
retryBackoffMs,
|
||||
new ConsumerCoordinator.DefaultOffsetCommitCallback(),
|
||||
|
|
@ -606,7 +605,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
this.subscriptions,
|
||||
metrics,
|
||||
metricGrpPrefix,
|
||||
metricsTags,
|
||||
this.time,
|
||||
this.retryBackoffMs);
|
||||
|
||||
|
|
|
|||
|
|
@ -14,7 +14,6 @@ package org.apache.kafka.clients.consumer.internals;
|
|||
|
||||
import org.apache.kafka.clients.ClientResponse;
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.errors.DisconnectException;
|
||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||
|
|
@ -107,7 +106,6 @@ public abstract class AbstractCoordinator implements Closeable {
|
|||
int heartbeatIntervalMs,
|
||||
Metrics metrics,
|
||||
String metricGrpPrefix,
|
||||
Map<String, String> metricTags,
|
||||
Time time,
|
||||
long retryBackoffMs) {
|
||||
this.client = client;
|
||||
|
|
@ -119,7 +117,7 @@ public abstract class AbstractCoordinator implements Closeable {
|
|||
this.sessionTimeoutMs = sessionTimeoutMs;
|
||||
this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds());
|
||||
this.heartbeatTask = new HeartbeatTask();
|
||||
this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
|
||||
this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix);
|
||||
this.retryBackoffMs = retryBackoffMs;
|
||||
}
|
||||
|
||||
|
|
@ -679,47 +677,39 @@ public abstract class AbstractCoordinator implements Closeable {
|
|||
public final Sensor joinLatency;
|
||||
public final Sensor syncLatency;
|
||||
|
||||
public GroupCoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
|
||||
public GroupCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) {
|
||||
this.metrics = metrics;
|
||||
this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
|
||||
|
||||
this.heartbeatLatency = metrics.sensor("heartbeat-latency");
|
||||
this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
|
||||
this.heartbeatLatency.add(metrics.metricName("heartbeat-response-time-max",
|
||||
this.metricGrpName,
|
||||
"The max time taken to receive a response to a heartbeat request",
|
||||
tags), new Max());
|
||||
this.heartbeatLatency.add(new MetricName("heartbeat-rate",
|
||||
"The max time taken to receive a response to a heartbeat request"), new Max());
|
||||
this.heartbeatLatency.add(metrics.metricName("heartbeat-rate",
|
||||
this.metricGrpName,
|
||||
"The average number of heartbeats per second",
|
||||
tags), new Rate(new Count()));
|
||||
"The average number of heartbeats per second"), new Rate(new Count()));
|
||||
|
||||
this.joinLatency = metrics.sensor("join-latency");
|
||||
this.joinLatency.add(new MetricName("join-time-avg",
|
||||
this.joinLatency.add(metrics.metricName("join-time-avg",
|
||||
this.metricGrpName,
|
||||
"The average time taken for a group rejoin",
|
||||
tags), new Avg());
|
||||
this.joinLatency.add(new MetricName("join-time-max",
|
||||
"The average time taken for a group rejoin"), new Avg());
|
||||
this.joinLatency.add(metrics.metricName("join-time-max",
|
||||
this.metricGrpName,
|
||||
"The max time taken for a group rejoin",
|
||||
tags), new Avg());
|
||||
this.joinLatency.add(new MetricName("join-rate",
|
||||
"The max time taken for a group rejoin"), new Avg());
|
||||
this.joinLatency.add(metrics.metricName("join-rate",
|
||||
this.metricGrpName,
|
||||
"The number of group joins per second",
|
||||
tags), new Rate(new Count()));
|
||||
"The number of group joins per second"), new Rate(new Count()));
|
||||
|
||||
this.syncLatency = metrics.sensor("sync-latency");
|
||||
this.syncLatency.add(new MetricName("sync-time-avg",
|
||||
this.syncLatency.add(metrics.metricName("sync-time-avg",
|
||||
this.metricGrpName,
|
||||
"The average time taken for a group sync",
|
||||
tags), new Avg());
|
||||
this.syncLatency.add(new MetricName("sync-time-max",
|
||||
"The average time taken for a group sync"), new Avg());
|
||||
this.syncLatency.add(metrics.metricName("sync-time-max",
|
||||
this.metricGrpName,
|
||||
"The max time taken for a group sync",
|
||||
tags), new Avg());
|
||||
this.syncLatency.add(new MetricName("sync-rate",
|
||||
"The max time taken for a group sync"), new Avg());
|
||||
this.syncLatency.add(metrics.metricName("sync-rate",
|
||||
this.metricGrpName,
|
||||
"The number of group syncs per second",
|
||||
tags), new Rate(new Count()));
|
||||
"The number of group syncs per second"), new Rate(new Count()));
|
||||
|
||||
Measurable lastHeartbeat =
|
||||
new Measurable() {
|
||||
|
|
@ -727,10 +717,9 @@ public abstract class AbstractCoordinator implements Closeable {
|
|||
return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
|
||||
}
|
||||
};
|
||||
metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
|
||||
metrics.addMetric(metrics.metricName("last-heartbeat-seconds-ago",
|
||||
this.metricGrpName,
|
||||
"The number of seconds since the last controller heartbeat",
|
||||
tags),
|
||||
"The number of seconds since the last controller heartbeat"),
|
||||
lastHeartbeat);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Assignment;
|
|||
import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Subscription;
|
||||
import org.apache.kafka.common.Cluster;
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.errors.GroupAuthorizationException;
|
||||
import org.apache.kafka.common.errors.TopicAuthorizationException;
|
||||
|
|
@ -82,7 +81,6 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
|||
SubscriptionState subscriptions,
|
||||
Metrics metrics,
|
||||
String metricGrpPrefix,
|
||||
Map<String, String> metricTags,
|
||||
Time time,
|
||||
long retryBackoffMs,
|
||||
OffsetCommitCallback defaultOffsetCommitCallback,
|
||||
|
|
@ -94,7 +92,6 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
|||
heartbeatIntervalMs,
|
||||
metrics,
|
||||
metricGrpPrefix,
|
||||
metricTags,
|
||||
time,
|
||||
retryBackoffMs);
|
||||
this.metadata = metadata;
|
||||
|
|
@ -109,7 +106,7 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
|||
addMetadataListener();
|
||||
|
||||
this.autoCommitTask = autoCommitEnabled ? new AutoCommitTask(autoCommitIntervalMs) : null;
|
||||
this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
|
||||
this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -639,23 +636,20 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
|||
|
||||
public final Sensor commitLatency;
|
||||
|
||||
public ConsumerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
|
||||
public ConsumerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) {
|
||||
this.metrics = metrics;
|
||||
this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
|
||||
|
||||
this.commitLatency = metrics.sensor("commit-latency");
|
||||
this.commitLatency.add(new MetricName("commit-latency-avg",
|
||||
this.commitLatency.add(metrics.metricName("commit-latency-avg",
|
||||
this.metricGrpName,
|
||||
"The average time taken for a commit request",
|
||||
tags), new Avg());
|
||||
this.commitLatency.add(new MetricName("commit-latency-max",
|
||||
"The average time taken for a commit request"), new Avg());
|
||||
this.commitLatency.add(metrics.metricName("commit-latency-max",
|
||||
this.metricGrpName,
|
||||
"The max time taken for a commit request",
|
||||
tags), new Max());
|
||||
this.commitLatency.add(new MetricName("commit-rate",
|
||||
"The max time taken for a commit request"), new Max());
|
||||
this.commitLatency.add(metrics.metricName("commit-rate",
|
||||
this.metricGrpName,
|
||||
"The number of commit calls per second",
|
||||
tags), new Rate(new Count()));
|
||||
"The number of commit calls per second"), new Rate(new Count()));
|
||||
|
||||
Measurable numParts =
|
||||
new Measurable() {
|
||||
|
|
@ -663,11 +657,9 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
|
|||
return subscriptions.assignedPartitions().size();
|
||||
}
|
||||
};
|
||||
metrics.addMetric(new MetricName("assigned-partitions",
|
||||
metrics.addMetric(metrics.metricName("assigned-partitions",
|
||||
this.metricGrpName,
|
||||
"The number of partitions currently assigned to this consumer",
|
||||
tags),
|
||||
numParts);
|
||||
"The number of partitions currently assigned to this consumer"), numParts);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
|
|||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||
import org.apache.kafka.common.Cluster;
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.PartitionInfo;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
|
|
@ -99,7 +98,6 @@ public class Fetcher<K, V> {
|
|||
SubscriptionState subscriptions,
|
||||
Metrics metrics,
|
||||
String metricGrpPrefix,
|
||||
Map<String, String> metricTags,
|
||||
Time time,
|
||||
long retryBackoffMs) {
|
||||
|
||||
|
|
@ -120,7 +118,7 @@ public class Fetcher<K, V> {
|
|||
this.unauthorizedTopics = new HashSet<>();
|
||||
this.recordTooLargePartitions = new HashMap<>();
|
||||
|
||||
this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix, metricTags);
|
||||
this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix);
|
||||
this.retryBackoffMs = retryBackoffMs;
|
||||
}
|
||||
|
||||
|
|
@ -656,64 +654,53 @@ public class Fetcher<K, V> {
|
|||
public final Sensor fetchThrottleTimeSensor;
|
||||
|
||||
|
||||
public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
|
||||
public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix) {
|
||||
this.metrics = metrics;
|
||||
this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics";
|
||||
|
||||
this.bytesFetched = metrics.sensor("bytes-fetched");
|
||||
this.bytesFetched.add(new MetricName("fetch-size-avg",
|
||||
this.bytesFetched.add(metrics.metricName("fetch-size-avg",
|
||||
this.metricGrpName,
|
||||
"The average number of bytes fetched per request",
|
||||
tags), new Avg());
|
||||
this.bytesFetched.add(new MetricName("fetch-size-max",
|
||||
"The average number of bytes fetched per request"), new Avg());
|
||||
this.bytesFetched.add(metrics.metricName("fetch-size-max",
|
||||
this.metricGrpName,
|
||||
"The maximum number of bytes fetched per request",
|
||||
tags), new Max());
|
||||
this.bytesFetched.add(new MetricName("bytes-consumed-rate",
|
||||
"The maximum number of bytes fetched per request"), new Max());
|
||||
this.bytesFetched.add(metrics.metricName("bytes-consumed-rate",
|
||||
this.metricGrpName,
|
||||
"The average number of bytes consumed per second",
|
||||
tags), new Rate());
|
||||
"The average number of bytes consumed per second"), new Rate());
|
||||
|
||||
this.recordsFetched = metrics.sensor("records-fetched");
|
||||
this.recordsFetched.add(new MetricName("records-per-request-avg",
|
||||
this.recordsFetched.add(metrics.metricName("records-per-request-avg",
|
||||
this.metricGrpName,
|
||||
"The average number of records in each request",
|
||||
tags), new Avg());
|
||||
this.recordsFetched.add(new MetricName("records-consumed-rate",
|
||||
"The average number of records in each request"), new Avg());
|
||||
this.recordsFetched.add(metrics.metricName("records-consumed-rate",
|
||||
this.metricGrpName,
|
||||
"The average number of records consumed per second",
|
||||
tags), new Rate());
|
||||
"The average number of records consumed per second"), new Rate());
|
||||
|
||||
this.fetchLatency = metrics.sensor("fetch-latency");
|
||||
this.fetchLatency.add(new MetricName("fetch-latency-avg",
|
||||
this.fetchLatency.add(metrics.metricName("fetch-latency-avg",
|
||||
this.metricGrpName,
|
||||
"The average time taken for a fetch request.",
|
||||
tags), new Avg());
|
||||
this.fetchLatency.add(new MetricName("fetch-latency-max",
|
||||
"The average time taken for a fetch request."), new Avg());
|
||||
this.fetchLatency.add(metrics.metricName("fetch-latency-max",
|
||||
this.metricGrpName,
|
||||
"The max time taken for any fetch request.",
|
||||
tags), new Max());
|
||||
this.fetchLatency.add(new MetricName("fetch-rate",
|
||||
"The max time taken for any fetch request."), new Max());
|
||||
this.fetchLatency.add(metrics.metricName("fetch-rate",
|
||||
this.metricGrpName,
|
||||
"The number of fetch requests per second.",
|
||||
tags), new Rate(new Count()));
|
||||
"The number of fetch requests per second."), new Rate(new Count()));
|
||||
|
||||
this.recordsFetchLag = metrics.sensor("records-lag");
|
||||
this.recordsFetchLag.add(new MetricName("records-lag-max",
|
||||
this.recordsFetchLag.add(metrics.metricName("records-lag-max",
|
||||
this.metricGrpName,
|
||||
"The maximum lag in terms of number of records for any partition in this window",
|
||||
tags), new Max());
|
||||
"The maximum lag in terms of number of records for any partition in this window"), new Max());
|
||||
|
||||
this.fetchThrottleTimeSensor = metrics.sensor("fetch-throttle-time");
|
||||
this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-avg",
|
||||
this.fetchThrottleTimeSensor.add(metrics.metricName("fetch-throttle-time-avg",
|
||||
this.metricGrpName,
|
||||
"The average throttle time in ms",
|
||||
tags), new Avg());
|
||||
"The average throttle time in ms"), new Avg());
|
||||
|
||||
this.fetchThrottleTimeSensor.add(new MetricName("fetch-throttle-time-max",
|
||||
this.fetchThrottleTimeSensor.add(metrics.metricName("fetch-throttle-time-max",
|
||||
this.metricGrpName,
|
||||
"The maximum throttle time in ms",
|
||||
tags), new Max());
|
||||
"The maximum throttle time in ms"), new Max());
|
||||
}
|
||||
|
||||
public void recordTopicFetchMetrics(String topic, int bytes, int records) {
|
||||
|
|
|
|||
|
|
@ -203,12 +203,14 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
this.producerConfig = config;
|
||||
this.time = new SystemTime();
|
||||
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
|
||||
TimeUnit.MILLISECONDS);
|
||||
clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG);
|
||||
if (clientId.length() <= 0)
|
||||
clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement();
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
metricTags.put("client-id", clientId);
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS)
|
||||
.tags(metricTags);
|
||||
List<MetricsReporter> reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG,
|
||||
MetricsReporter.class);
|
||||
reporters.add(new JmxReporter(JMX_PREFIX));
|
||||
|
|
@ -256,21 +258,18 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
this.requestTimeoutMs = config.getInt(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG);
|
||||
}
|
||||
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
metricTags.put("client-id", clientId);
|
||||
this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG),
|
||||
this.totalMemorySize,
|
||||
this.compressionType,
|
||||
config.getLong(ProducerConfig.LINGER_MS_CONFIG),
|
||||
retryBackoffMs,
|
||||
metrics,
|
||||
time,
|
||||
metricTags);
|
||||
time);
|
||||
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
|
||||
this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());
|
||||
ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values());
|
||||
NetworkClient client = new NetworkClient(
|
||||
new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", metricTags, channelBuilder),
|
||||
new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", channelBuilder),
|
||||
this.metadata,
|
||||
clientId,
|
||||
config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION),
|
||||
|
|
|
|||
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.clients.producer.internals;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Deque;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
|
@ -62,9 +61,8 @@ public final class BufferPool {
|
|||
* @param metrics instance of Metrics
|
||||
* @param time time instance
|
||||
* @param metricGrpName logical group name for metrics
|
||||
* @param metricTags additional key/val attributes for metrics
|
||||
*/
|
||||
public BufferPool(long memory, int poolableSize, Metrics metrics, Time time , String metricGrpName , Map<String, String> metricTags) {
|
||||
public BufferPool(long memory, int poolableSize, Metrics metrics, Time time, String metricGrpName) {
|
||||
this.poolableSize = poolableSize;
|
||||
this.lock = new ReentrantLock();
|
||||
this.free = new ArrayDeque<ByteBuffer>();
|
||||
|
|
@ -74,10 +72,9 @@ public final class BufferPool {
|
|||
this.metrics = metrics;
|
||||
this.time = time;
|
||||
this.waitTime = this.metrics.sensor("bufferpool-wait-time");
|
||||
MetricName metricName = new MetricName("bufferpool-wait-ratio",
|
||||
metricGrpName,
|
||||
"The fraction of time an appender waits for space allocation.",
|
||||
metricTags);
|
||||
MetricName metricName = metrics.metricName("bufferpool-wait-ratio",
|
||||
metricGrpName,
|
||||
"The fraction of time an appender waits for space allocation.");
|
||||
this.waitTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -85,7 +85,6 @@ public final class RecordAccumulator {
|
|||
* exhausting all retries in a short period of time.
|
||||
* @param metrics The metrics
|
||||
* @param time The time instance to use
|
||||
* @param metricTags additional key/value attributes of the metric
|
||||
*/
|
||||
public RecordAccumulator(int batchSize,
|
||||
long totalSize,
|
||||
|
|
@ -93,8 +92,7 @@ public final class RecordAccumulator {
|
|||
long lingerMs,
|
||||
long retryBackoffMs,
|
||||
Metrics metrics,
|
||||
Time time,
|
||||
Map<String, String> metricTags) {
|
||||
Time time) {
|
||||
this.drainIndex = 0;
|
||||
this.closed = false;
|
||||
this.flushesInProgress = new AtomicInteger(0);
|
||||
|
|
@ -105,14 +103,14 @@ public final class RecordAccumulator {
|
|||
this.retryBackoffMs = retryBackoffMs;
|
||||
this.batches = new CopyOnWriteMap<TopicPartition, Deque<RecordBatch>>();
|
||||
String metricGrpName = "producer-metrics";
|
||||
this.free = new BufferPool(totalSize, batchSize, metrics, time , metricGrpName , metricTags);
|
||||
this.free = new BufferPool(totalSize, batchSize, metrics, time, metricGrpName);
|
||||
this.incomplete = new IncompleteRecordBatches();
|
||||
this.time = time;
|
||||
registerMetrics(metrics, metricGrpName, metricTags);
|
||||
registerMetrics(metrics, metricGrpName);
|
||||
}
|
||||
|
||||
private void registerMetrics(Metrics metrics, String metricGrpName, Map<String, String> metricTags) {
|
||||
MetricName metricName = new MetricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records", metricTags);
|
||||
private void registerMetrics(Metrics metrics, String metricGrpName) {
|
||||
MetricName metricName = metrics.metricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records");
|
||||
Measurable waitingThreads = new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return free.queued();
|
||||
|
|
@ -120,7 +118,7 @@ public final class RecordAccumulator {
|
|||
};
|
||||
metrics.addMetric(metricName, waitingThreads);
|
||||
|
||||
metricName = new MetricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).", metricTags);
|
||||
metricName = metrics.metricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).");
|
||||
Measurable totalBytes = new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return free.totalMemory();
|
||||
|
|
@ -128,7 +126,7 @@ public final class RecordAccumulator {
|
|||
};
|
||||
metrics.addMetric(metricName, totalBytes);
|
||||
|
||||
metricName = new MetricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).", metricTags);
|
||||
metricName = metrics.metricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).");
|
||||
Measurable availableBytes = new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return free.availableMemory();
|
||||
|
|
@ -137,7 +135,7 @@ public final class RecordAccumulator {
|
|||
metrics.addMetric(metricName, availableBytes);
|
||||
|
||||
Sensor bufferExhaustedRecordSensor = metrics.sensor("buffer-exhausted-records");
|
||||
metricName = new MetricName("buffer-exhausted-rate", metricGrpName, "The average per-second number of record sends that are dropped due to buffer exhaustion", metricTags);
|
||||
metricName = metrics.metricName("buffer-exhausted-rate", metricGrpName, "The average per-second number of record sends that are dropped due to buffer exhaustion");
|
||||
bufferExhaustedRecordSensor.add(metricName, new Rate());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -367,65 +367,63 @@ public class Sender implements Runnable {
|
|||
|
||||
public SenderMetrics(Metrics metrics) {
|
||||
this.metrics = metrics;
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
metricTags.put("client-id", clientId);
|
||||
String metricGrpName = "producer-metrics";
|
||||
|
||||
this.batchSizeSensor = metrics.sensor("batch-size");
|
||||
MetricName m = new MetricName("batch-size-avg", metricGrpName, "The average number of bytes sent per partition per-request.", metricTags);
|
||||
MetricName m = metrics.metricName("batch-size-avg", metricGrpName, "The average number of bytes sent per partition per-request.");
|
||||
this.batchSizeSensor.add(m, new Avg());
|
||||
m = new MetricName("batch-size-max", metricGrpName, "The max number of bytes sent per partition per-request.", metricTags);
|
||||
m = metrics.metricName("batch-size-max", metricGrpName, "The max number of bytes sent per partition per-request.");
|
||||
this.batchSizeSensor.add(m, new Max());
|
||||
|
||||
this.compressionRateSensor = metrics.sensor("compression-rate");
|
||||
m = new MetricName("compression-rate-avg", metricGrpName, "The average compression rate of record batches.", metricTags);
|
||||
m = metrics.metricName("compression-rate-avg", metricGrpName, "The average compression rate of record batches.");
|
||||
this.compressionRateSensor.add(m, new Avg());
|
||||
|
||||
this.queueTimeSensor = metrics.sensor("queue-time");
|
||||
m = new MetricName("record-queue-time-avg", metricGrpName, "The average time in ms record batches spent in the record accumulator.", metricTags);
|
||||
m = metrics.metricName("record-queue-time-avg", metricGrpName, "The average time in ms record batches spent in the record accumulator.");
|
||||
this.queueTimeSensor.add(m, new Avg());
|
||||
m = new MetricName("record-queue-time-max", metricGrpName, "The maximum time in ms record batches spent in the record accumulator.", metricTags);
|
||||
m = metrics.metricName("record-queue-time-max", metricGrpName, "The maximum time in ms record batches spent in the record accumulator.");
|
||||
this.queueTimeSensor.add(m, new Max());
|
||||
|
||||
this.requestTimeSensor = metrics.sensor("request-time");
|
||||
m = new MetricName("request-latency-avg", metricGrpName, "The average request latency in ms", metricTags);
|
||||
m = metrics.metricName("request-latency-avg", metricGrpName, "The average request latency in ms");
|
||||
this.requestTimeSensor.add(m, new Avg());
|
||||
m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags);
|
||||
m = metrics.metricName("request-latency-max", metricGrpName, "The maximum request latency in ms");
|
||||
this.requestTimeSensor.add(m, new Max());
|
||||
|
||||
this.produceThrottleTimeSensor = metrics.sensor("produce-throttle-time");
|
||||
m = new MetricName("produce-throttle-time-avg", metricGrpName, "The average throttle time in ms", metricTags);
|
||||
m = metrics.metricName("produce-throttle-time-avg", metricGrpName, "The average throttle time in ms");
|
||||
this.produceThrottleTimeSensor.add(m, new Avg());
|
||||
m = new MetricName("produce-throttle-time-max", metricGrpName, "The maximum throttle time in ms", metricTags);
|
||||
m = metrics.metricName("produce-throttle-time-max", metricGrpName, "The maximum throttle time in ms");
|
||||
this.produceThrottleTimeSensor.add(m, new Max());
|
||||
|
||||
this.recordsPerRequestSensor = metrics.sensor("records-per-request");
|
||||
m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags);
|
||||
m = metrics.metricName("record-send-rate", metricGrpName, "The average number of records sent per second.");
|
||||
this.recordsPerRequestSensor.add(m, new Rate());
|
||||
m = new MetricName("records-per-request-avg", metricGrpName, "The average number of records per request.", metricTags);
|
||||
m = metrics.metricName("records-per-request-avg", metricGrpName, "The average number of records per request.");
|
||||
this.recordsPerRequestSensor.add(m, new Avg());
|
||||
|
||||
this.retrySensor = metrics.sensor("record-retries");
|
||||
m = new MetricName("record-retry-rate", metricGrpName, "The average per-second number of retried record sends", metricTags);
|
||||
m = metrics.metricName("record-retry-rate", metricGrpName, "The average per-second number of retried record sends");
|
||||
this.retrySensor.add(m, new Rate());
|
||||
|
||||
this.errorSensor = metrics.sensor("errors");
|
||||
m = new MetricName("record-error-rate", metricGrpName, "The average per-second number of record sends that resulted in errors", metricTags);
|
||||
m = metrics.metricName("record-error-rate", metricGrpName, "The average per-second number of record sends that resulted in errors");
|
||||
this.errorSensor.add(m, new Rate());
|
||||
|
||||
this.maxRecordSizeSensor = metrics.sensor("record-size-max");
|
||||
m = new MetricName("record-size-max", metricGrpName, "The maximum record size", metricTags);
|
||||
m = metrics.metricName("record-size-max", metricGrpName, "The maximum record size");
|
||||
this.maxRecordSizeSensor.add(m, new Max());
|
||||
m = new MetricName("record-size-avg", metricGrpName, "The average record size", metricTags);
|
||||
m = metrics.metricName("record-size-avg", metricGrpName, "The average record size");
|
||||
this.maxRecordSizeSensor.add(m, new Avg());
|
||||
|
||||
m = new MetricName("requests-in-flight", metricGrpName, "The current number of in-flight requests awaiting a response.", metricTags);
|
||||
m = metrics.metricName("requests-in-flight", metricGrpName, "The current number of in-flight requests awaiting a response.");
|
||||
this.metrics.addMetric(m, new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return client.inFlightRequestCount();
|
||||
}
|
||||
});
|
||||
m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags);
|
||||
m = metrics.metricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.");
|
||||
metrics.addMetric(m, new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return (now - metadata.lastSuccessfulUpdate()) / 1000.0;
|
||||
|
|
@ -440,32 +438,31 @@ public class Sender implements Runnable {
|
|||
Sensor topicRecordCount = this.metrics.getSensor(topicRecordsCountName);
|
||||
if (topicRecordCount == null) {
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
metricTags.put("client-id", clientId);
|
||||
metricTags.put("topic", topic);
|
||||
String metricGrpName = "producer-topic-metrics";
|
||||
|
||||
topicRecordCount = this.metrics.sensor(topicRecordsCountName);
|
||||
MetricName m = new MetricName("record-send-rate", metricGrpName , metricTags);
|
||||
MetricName m = this.metrics.metricName("record-send-rate", metricGrpName, metricTags);
|
||||
topicRecordCount.add(m, new Rate());
|
||||
|
||||
String topicByteRateName = "topic." + topic + ".bytes";
|
||||
Sensor topicByteRate = this.metrics.sensor(topicByteRateName);
|
||||
m = new MetricName("byte-rate", metricGrpName , metricTags);
|
||||
m = this.metrics.metricName("byte-rate", metricGrpName, metricTags);
|
||||
topicByteRate.add(m, new Rate());
|
||||
|
||||
String topicCompressionRateName = "topic." + topic + ".compression-rate";
|
||||
Sensor topicCompressionRate = this.metrics.sensor(topicCompressionRateName);
|
||||
m = new MetricName("compression-rate", metricGrpName , metricTags);
|
||||
m = this.metrics.metricName("compression-rate", metricGrpName, metricTags);
|
||||
topicCompressionRate.add(m, new Avg());
|
||||
|
||||
String topicRetryName = "topic." + topic + ".record-retries";
|
||||
Sensor topicRetrySensor = this.metrics.sensor(topicRetryName);
|
||||
m = new MetricName("record-retry-rate", metricGrpName , metricTags);
|
||||
m = this.metrics.metricName("record-retry-rate", metricGrpName, metricTags);
|
||||
topicRetrySensor.add(m, new Rate());
|
||||
|
||||
String topicErrorName = "topic." + topic + ".record-errors";
|
||||
Sensor topicErrorSensor = this.metrics.sensor(topicErrorName);
|
||||
m = new MetricName("record-error-rate", metricGrpName , metricTags);
|
||||
m = this.metrics.metricName("record-error-rate", metricGrpName, metricTags);
|
||||
topicErrorSensor.add(m, new Rate());
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import java.util.Map;
|
|||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
/**
|
||||
* The <code>MetricName</code> class encapsulates a metric's name, logical group and its related attributes
|
||||
* The <code>MetricName</code> class encapsulates a metric's name, logical group and its related attributes. It should be constructed using metrics.MetricName(...).
|
||||
* <p>
|
||||
* This class captures the following parameters
|
||||
* <pre>
|
||||
|
|
@ -31,23 +31,27 @@ import org.apache.kafka.common.utils.Utils;
|
|||
* <p>
|
||||
* Ex: standard JMX MBean can be constructed like <b>domainName:type=group,key1=val1,key2=val2</b>
|
||||
* <p>
|
||||
*
|
||||
* Usage looks something like this:
|
||||
* <pre>{@code
|
||||
* // set up metrics:
|
||||
* Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
|
||||
* Sensor sensor = metrics.sensor("message-sizes");
|
||||
*
|
||||
* Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
* metricTags.put("client-id", "producer-1");
|
||||
* metricTags.put("topic", "topic");
|
||||
*
|
||||
* MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
|
||||
* MetricConfig metricConfig = new MetricConfig().tags(metricTags);
|
||||
* Metrics metrics = new Metrics(metricConfig); // this is the global repository of metrics and sensors
|
||||
*
|
||||
* Sensor sensor = metrics.sensor("message-sizes");
|
||||
*
|
||||
* MetricName metricName = metrics.metricName("message-size-avg", "producer-metrics", "average message size");
|
||||
* sensor.add(metricName, new Avg());
|
||||
*
|
||||
* metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
|
||||
* metricName = metrics.metricName("message-size-max", "producer-metrics");
|
||||
* sensor.add(metricName, new Max());
|
||||
*
|
||||
* metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
|
||||
* metricName = metrics.metricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
|
||||
* sensor.add(metricName, new Min());
|
||||
*
|
||||
* // as messages are sent we record the sizes
|
||||
|
|
@ -63,6 +67,8 @@ public final class MetricName {
|
|||
private int hash = 0;
|
||||
|
||||
/**
|
||||
* Please create MetricName by method {@link org.apache.kafka.common.metrics.Metrics#metricName(String, String, String, Map<String, String>)}
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
|
|
@ -76,11 +82,15 @@ public final class MetricName {
|
|||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method will be removed in a future release.
|
||||
* Please create MetricName by method {@link org.apache.kafka.common.metrics.Metrics#metricName(String, String, String, String...)}
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
* @param keyValue additional key/value attributes of the metric (must come in pairs)
|
||||
*/
|
||||
@Deprecated
|
||||
public MetricName(String name, String group, String description, String... keyValue) {
|
||||
this(name, group, description, getTags(keyValue));
|
||||
}
|
||||
|
|
@ -97,27 +107,39 @@ public final class MetricName {
|
|||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method will be removed in a future release.
|
||||
* Please create MetricName by method {@link org.apache.kafka.common.metrics.Metrics#metricName(String, String, Map<String, String>)}
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param tags key/value attributes of the metric
|
||||
*/
|
||||
@Deprecated
|
||||
public MetricName(String name, String group, Map<String, String> tags) {
|
||||
this(name, group, "", tags);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method will be removed in a future release.
|
||||
* Please create MetricName by method {@link org.apache.kafka.common.metrics.Metrics#metricName(String, String, String)}
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
*/
|
||||
@Deprecated
|
||||
public MetricName(String name, String group, String description) {
|
||||
this(name, group, description, new HashMap<String, String>());
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated This method will be removed in a future release.
|
||||
* Please create MetricName by method {@link org.apache.kafka.common.metrics.Metrics#metricName(String, String)}
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
*/
|
||||
@Deprecated
|
||||
public MetricName(String name, String group) {
|
||||
this(name, group, "", new HashMap<String, String>());
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.kafka.common.metrics;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
|
|
@ -28,6 +30,7 @@ public class MetricConfig {
|
|||
private long eventWindow;
|
||||
private long timeWindowMs;
|
||||
private TimeUnit unit;
|
||||
private Map<String, String> tags;
|
||||
|
||||
public MetricConfig() {
|
||||
super();
|
||||
|
|
@ -36,6 +39,7 @@ public class MetricConfig {
|
|||
this.eventWindow = Long.MAX_VALUE;
|
||||
this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS);
|
||||
this.unit = TimeUnit.SECONDS;
|
||||
this.tags = new LinkedHashMap<>();
|
||||
}
|
||||
|
||||
public Quota quota() {
|
||||
|
|
@ -65,6 +69,15 @@ public class MetricConfig {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Map<String, String> tags() {
|
||||
return this.tags;
|
||||
}
|
||||
|
||||
public MetricConfig tags(Map<String, String> tags) {
|
||||
this.tags = tags;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int samples() {
|
||||
return this.samples;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,6 +15,8 @@ package org.apache.kafka.common.metrics;
|
|||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
|
@ -81,6 +83,15 @@ public class Metrics implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a metrics repository with no metric reporters and the given default configuration.
|
||||
* Expiration of Sensors is disabled.
|
||||
*/
|
||||
public Metrics(MetricConfig defaultConfig, Time time) {
|
||||
this(defaultConfig, new ArrayList<MetricsReporter>(0), time);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a metrics repository with no reporters and the given default config. This config will be used for any
|
||||
* metric that doesn't override its own config. Expiration of Sensors is disabled.
|
||||
* @param defaultConfig The default config to use for all metrics that don't override their config
|
||||
|
|
@ -130,6 +141,90 @@ public class Metrics implements Closeable {
|
|||
} else {
|
||||
this.metricsScheduler = null;
|
||||
}
|
||||
|
||||
addMetric(metricName("count", "kafka-metrics-count", "total number of registered metrics"),
|
||||
new Measurable() {
|
||||
@Override
|
||||
public double measure(MetricConfig config, long now) {
|
||||
return metrics.size();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a MetricName with the given name, group, description and tags, plus default tags specified in the metric
|
||||
* configuration. Tag in tags takes precedence if the same tag key is specified in the default metric configuration.
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
* @param tags additional key/value attributes of the metric
|
||||
*/
|
||||
public MetricName metricName(String name, String group, String description, Map<String, String> tags) {
|
||||
Map<String, String> combinedTag = new LinkedHashMap<>(config.tags());
|
||||
combinedTag.putAll(tags);
|
||||
return new MetricName(name, group, description, combinedTag);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a MetricName with the given name, group, description, and default tags
|
||||
* specified in the metric configuration.
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
*/
|
||||
public MetricName metricName(String name, String group, String description) {
|
||||
return metricName(name, group, description, new HashMap<String, String>());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a MetricName with the given name, group and default tags specified in the metric configuration.
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
*/
|
||||
public MetricName metricName(String name, String group) {
|
||||
return metricName(name, group, "", new HashMap<String, String>());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a MetricName with the given name, group, description, and keyValue as tags, plus default tags specified in the metric
|
||||
* configuration. Tag in keyValue takes precedence if the same tag key is specified in the default metric configuration.
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param description A human-readable description to include in the metric
|
||||
* @param keyValue additional key/value attributes of the metric (must come in pairs)
|
||||
*/
|
||||
public MetricName metricName(String name, String group, String description, String... keyValue) {
|
||||
return metricName(name, group, description, getTags(keyValue));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a MetricName with the given name, group and tags, plus default tags specified in the metric
|
||||
* configuration. Tag in tags takes precedence if the same tag key is specified in the default metric configuration.
|
||||
*
|
||||
* @param name The name of the metric
|
||||
* @param group logical group name of the metrics to which this metric belongs
|
||||
* @param tags key/value attributes of the metric
|
||||
*/
|
||||
public MetricName metricName(String name, String group, Map<String, String> tags) {
|
||||
return metricName(name, group, "", tags);
|
||||
}
|
||||
|
||||
private static Map<String, String> getTags(String... keyValue) {
|
||||
if ((keyValue.length % 2) != 0)
|
||||
throw new IllegalArgumentException("keyValue needs to be specified in pairs");
|
||||
Map<String, String> tags = new HashMap<String, String>();
|
||||
|
||||
for (int i = 0; i < keyValue.length; i += 2)
|
||||
tags.put(keyValue[i], keyValue[i + 1]);
|
||||
return tags;
|
||||
}
|
||||
|
||||
public MetricConfig config() {
|
||||
return config;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -118,8 +118,8 @@ public class Selector implements Selectable {
|
|||
this.metricsPerConnection = metricsPerConnection;
|
||||
}
|
||||
|
||||
public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, Map<String, String> metricTags, ChannelBuilder channelBuilder) {
|
||||
this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, metricTags, true, channelBuilder);
|
||||
public Selector(long connectionMaxIdleMS, Metrics metrics, Time time, String metricGrpPrefix, ChannelBuilder channelBuilder) {
|
||||
this(NetworkReceive.UNLIMITED, connectionMaxIdleMS, metrics, time, metricGrpPrefix, new HashMap<String, String>(), true, channelBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -568,48 +568,48 @@ public class Selector implements Selectable {
|
|||
}
|
||||
|
||||
this.connectionClosed = sensor("connections-closed:" + tagsSuffix.toString());
|
||||
MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
|
||||
MetricName metricName = metrics.metricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags);
|
||||
this.connectionClosed.add(metricName, new Rate());
|
||||
|
||||
this.connectionCreated = sensor("connections-created:" + tagsSuffix.toString());
|
||||
metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
|
||||
metricName = metrics.metricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags);
|
||||
this.connectionCreated.add(metricName, new Rate());
|
||||
|
||||
this.bytesTransferred = sensor("bytes-sent-received:" + tagsSuffix.toString());
|
||||
metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
|
||||
metricName = metrics.metricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags);
|
||||
bytesTransferred.add(metricName, new Rate(new Count()));
|
||||
|
||||
this.bytesSent = sensor("bytes-sent:" + tagsSuffix.toString(), bytesTransferred);
|
||||
metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
|
||||
metricName = metrics.metricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags);
|
||||
this.bytesSent.add(metricName, new Rate());
|
||||
metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
|
||||
metricName = metrics.metricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags);
|
||||
this.bytesSent.add(metricName, new Rate(new Count()));
|
||||
metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags);
|
||||
metricName = metrics.metricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags);
|
||||
this.bytesSent.add(metricName, new Avg());
|
||||
metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
|
||||
metricName = metrics.metricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags);
|
||||
this.bytesSent.add(metricName, new Max());
|
||||
|
||||
this.bytesReceived = sensor("bytes-received:" + tagsSuffix.toString(), bytesTransferred);
|
||||
metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
|
||||
metricName = metrics.metricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags);
|
||||
this.bytesReceived.add(metricName, new Rate());
|
||||
metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
|
||||
metricName = metrics.metricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags);
|
||||
this.bytesReceived.add(metricName, new Rate(new Count()));
|
||||
|
||||
this.selectTime = sensor("select-time:" + tagsSuffix.toString());
|
||||
metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
|
||||
metricName = metrics.metricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags);
|
||||
this.selectTime.add(metricName, new Rate(new Count()));
|
||||
metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
|
||||
metricName = metrics.metricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags);
|
||||
this.selectTime.add(metricName, new Avg());
|
||||
metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
|
||||
metricName = metrics.metricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags);
|
||||
this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
|
||||
|
||||
this.ioTime = sensor("io-time:" + tagsSuffix.toString());
|
||||
metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
|
||||
metricName = metrics.metricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags);
|
||||
this.ioTime.add(metricName, new Avg());
|
||||
metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
|
||||
metricName = metrics.metricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags);
|
||||
this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS));
|
||||
|
||||
metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags);
|
||||
metricName = metrics.metricName("connection-count", metricGrpName, "The current number of active connections.", metricTags);
|
||||
topLevelMetricNames.add(metricName);
|
||||
this.metrics.addMetric(metricName, new Measurable() {
|
||||
public double measure(MetricConfig config, long now) {
|
||||
|
|
@ -637,27 +637,27 @@ public class Selector implements Selectable {
|
|||
tags.put("node-id", "node-" + connectionId);
|
||||
|
||||
nodeRequest = sensor(nodeRequestName);
|
||||
MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags);
|
||||
MetricName metricName = metrics.metricName("outgoing-byte-rate", metricGrpName, tags);
|
||||
nodeRequest.add(metricName, new Rate());
|
||||
metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags);
|
||||
metricName = metrics.metricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags);
|
||||
nodeRequest.add(metricName, new Rate(new Count()));
|
||||
metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags);
|
||||
metricName = metrics.metricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags);
|
||||
nodeRequest.add(metricName, new Avg());
|
||||
metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
|
||||
metricName = metrics.metricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags);
|
||||
nodeRequest.add(metricName, new Max());
|
||||
|
||||
String nodeResponseName = "node-" + connectionId + ".bytes-received";
|
||||
Sensor nodeResponse = sensor(nodeResponseName);
|
||||
metricName = new MetricName("incoming-byte-rate", metricGrpName, tags);
|
||||
metricName = metrics.metricName("incoming-byte-rate", metricGrpName, tags);
|
||||
nodeResponse.add(metricName, new Rate());
|
||||
metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
|
||||
metricName = metrics.metricName("response-rate", metricGrpName, "The average number of responses received per second.", tags);
|
||||
nodeResponse.add(metricName, new Rate(new Count()));
|
||||
|
||||
String nodeTimeName = "node-" + connectionId + ".latency";
|
||||
Sensor nodeRequestTime = sensor(nodeTimeName);
|
||||
metricName = new MetricName("request-latency-avg", metricGrpName, tags);
|
||||
metricName = metrics.metricName("request-latency-avg", metricGrpName, tags);
|
||||
nodeRequestTime.add(metricName, new Avg());
|
||||
metricName = new MetricName("request-latency-max", metricGrpName, tags);
|
||||
metricName = metrics.metricName("request-latency-max", metricGrpName, tags);
|
||||
nodeRequestTime.add(metricName, new Max());
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -60,7 +60,6 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
|
@ -89,7 +88,6 @@ public class ConsumerCoordinatorTest {
|
|||
private SubscriptionState subscriptions;
|
||||
private Metadata metadata;
|
||||
private Metrics metrics;
|
||||
private Map<String, String> metricTags = new LinkedHashMap<>();
|
||||
private ConsumerNetworkClient consumerClient;
|
||||
private MockRebalanceListener rebalanceListener;
|
||||
private MockCommitCallback defaultOffsetCommitCallback;
|
||||
|
|
@ -109,7 +107,6 @@ public class ConsumerCoordinatorTest {
|
|||
this.partitionAssignor.clear();
|
||||
|
||||
client.setNode(node);
|
||||
|
||||
this.coordinator = buildCoordinator(metrics, assignors);
|
||||
}
|
||||
|
||||
|
|
@ -912,7 +909,6 @@ public class ConsumerCoordinatorTest {
|
|||
subscriptions,
|
||||
metrics,
|
||||
"consumer" + groupId,
|
||||
metricTags,
|
||||
time,
|
||||
retryBackoffMs,
|
||||
defaultOffsetCommitCallback,
|
||||
|
|
|
|||
|
|
@ -54,7 +54,6 @@ import java.nio.ByteBuffer;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
|
@ -83,7 +82,6 @@ public class FetcherTest {
|
|||
private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
|
||||
private SubscriptionState subscriptionsNoAutoReset = new SubscriptionState(OffsetResetStrategy.NONE);
|
||||
private Metrics metrics = new Metrics(time);
|
||||
private Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
private static final double EPSILON = 0.0001;
|
||||
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
|
||||
|
||||
|
|
@ -484,8 +482,8 @@ public class FetcherTest {
|
|||
}
|
||||
|
||||
Map<MetricName, KafkaMetric> allMetrics = metrics.metrics();
|
||||
KafkaMetric avgMetric = allMetrics.get(new MetricName("fetch-throttle-time-avg", metricGroup, "", metricTags));
|
||||
KafkaMetric maxMetric = allMetrics.get(new MetricName("fetch-throttle-time-max", metricGroup, "", metricTags));
|
||||
KafkaMetric avgMetric = allMetrics.get(metrics.metricName("fetch-throttle-time-avg", metricGroup, ""));
|
||||
KafkaMetric maxMetric = allMetrics.get(metrics.metricName("fetch-throttle-time-max", metricGroup, ""));
|
||||
assertEquals(200, avgMetric.value(), EPSILON);
|
||||
assertEquals(300, maxMetric.value(), EPSILON);
|
||||
}
|
||||
|
|
@ -527,7 +525,6 @@ public class FetcherTest {
|
|||
subscriptions,
|
||||
metrics,
|
||||
"consumer" + groupId,
|
||||
metricTags,
|
||||
time,
|
||||
retryBackoffMs);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,9 +25,7 @@ import org.junit.Test;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
|
|
@ -38,7 +36,6 @@ public class BufferPoolTest {
|
|||
private Metrics metrics = new Metrics(time);
|
||||
private final long maxBlockTimeMs = 2000;
|
||||
String metricGroup = "TestMetrics";
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
|
||||
@After
|
||||
public void teardown() {
|
||||
|
|
@ -52,7 +49,7 @@ public class BufferPoolTest {
|
|||
public void testSimple() throws Exception {
|
||||
long totalMemory = 64 * 1024;
|
||||
int size = 1024;
|
||||
BufferPool pool = new BufferPool(totalMemory, size, metrics, time, metricGroup, metricTags);
|
||||
BufferPool pool = new BufferPool(totalMemory, size, metrics, time, metricGroup);
|
||||
ByteBuffer buffer = pool.allocate(size, maxBlockTimeMs);
|
||||
assertEquals("Buffer size should equal requested size.", size, buffer.limit());
|
||||
assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory());
|
||||
|
|
@ -79,7 +76,7 @@ public class BufferPoolTest {
|
|||
*/
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testCantAllocateMoreMemoryThanWeHave() throws Exception {
|
||||
BufferPool pool = new BufferPool(1024, 512, metrics, time, metricGroup, metricTags);
|
||||
BufferPool pool = new BufferPool(1024, 512, metrics, time, metricGroup);
|
||||
ByteBuffer buffer = pool.allocate(1024, maxBlockTimeMs);
|
||||
assertEquals(1024, buffer.limit());
|
||||
pool.deallocate(buffer);
|
||||
|
|
@ -91,7 +88,7 @@ public class BufferPoolTest {
|
|||
*/
|
||||
@Test
|
||||
public void testDelayedAllocation() throws Exception {
|
||||
BufferPool pool = new BufferPool(5 * 1024, 1024, metrics, time, metricGroup, metricTags);
|
||||
BufferPool pool = new BufferPool(5 * 1024, 1024, metrics, time, metricGroup);
|
||||
ByteBuffer buffer = pool.allocate(1024, maxBlockTimeMs);
|
||||
CountDownLatch doDealloc = asyncDeallocate(pool, buffer);
|
||||
CountDownLatch allocation = asyncAllocate(pool, 5 * 1024);
|
||||
|
|
@ -140,7 +137,7 @@ public class BufferPoolTest {
|
|||
*/
|
||||
@Test
|
||||
public void testBlockTimeout() throws Exception {
|
||||
BufferPool pool = new BufferPool(2, 1, metrics, time, metricGroup, metricTags);
|
||||
BufferPool pool = new BufferPool(2, 1, metrics, time, metricGroup);
|
||||
pool.allocate(1, maxBlockTimeMs);
|
||||
try {
|
||||
pool.allocate(2, maxBlockTimeMs);
|
||||
|
|
@ -159,7 +156,7 @@ public class BufferPoolTest {
|
|||
final int iterations = 50000;
|
||||
final int poolableSize = 1024;
|
||||
final long totalMemory = numThreads / 2 * poolableSize;
|
||||
final BufferPool pool = new BufferPool(totalMemory, poolableSize, metrics, time, metricGroup, metricTags);
|
||||
final BufferPool pool = new BufferPool(totalMemory, poolableSize, metrics, time, metricGroup);
|
||||
List<StressTestThread> threads = new ArrayList<StressTestThread>();
|
||||
for (int i = 0; i < numThreads; i++)
|
||||
threads.add(new StressTestThread(pool, iterations));
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
@ -65,7 +64,6 @@ public class RecordAccumulatorTest {
|
|||
private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value);
|
||||
private Cluster cluster = new Cluster(Arrays.asList(node1, node2), Arrays.asList(part1, part2, part3), Collections.<String>emptySet());
|
||||
private Metrics metrics = new Metrics(time);
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
private final long maxBlockTimeMs = 1000;
|
||||
|
||||
@After
|
||||
|
|
@ -76,7 +74,7 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testFull() throws Exception {
|
||||
long now = time.milliseconds();
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time);
|
||||
int appends = 1024 / msgSize;
|
||||
for (int i = 0; i < appends; i++) {
|
||||
accum.append(tp1, key, value, null, maxBlockTimeMs);
|
||||
|
|
@ -100,7 +98,7 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testAppendLarge() throws Exception {
|
||||
int batchSize = 512;
|
||||
RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
|
||||
accum.append(tp1, key, new byte[2 * batchSize], null, maxBlockTimeMs);
|
||||
assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
|
||||
}
|
||||
|
|
@ -108,7 +106,7 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testLinger() throws Exception {
|
||||
long lingerMs = 10L;
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
|
||||
accum.append(tp1, key, value, null, maxBlockTimeMs);
|
||||
assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
|
||||
time.sleep(10);
|
||||
|
|
@ -126,7 +124,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
@Test
|
||||
public void testPartialDrain() throws Exception {
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time);
|
||||
int appends = 1024 / msgSize + 1;
|
||||
List<TopicPartition> partitions = asList(tp1, tp2);
|
||||
for (TopicPartition tp : partitions) {
|
||||
|
|
@ -145,7 +143,7 @@ public class RecordAccumulatorTest {
|
|||
final int numThreads = 5;
|
||||
final int msgs = 10000;
|
||||
final int numParts = 2;
|
||||
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time, metricTags);
|
||||
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
|
||||
List<Thread> threads = new ArrayList<Thread>();
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
threads.add(new Thread() {
|
||||
|
|
@ -185,7 +183,7 @@ public class RecordAccumulatorTest {
|
|||
public void testNextReadyCheckDelay() throws Exception {
|
||||
// Next check time will use lingerMs since this test won't trigger any retries/backoff
|
||||
long lingerMs = 10L;
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
|
||||
// Just short of going over the limit so we trigger linger time
|
||||
int appends = 1024 / msgSize;
|
||||
|
||||
|
|
@ -219,7 +217,7 @@ public class RecordAccumulatorTest {
|
|||
public void testRetryBackoff() throws Exception {
|
||||
long lingerMs = Long.MAX_VALUE / 4;
|
||||
long retryBackoffMs = Long.MAX_VALUE / 2;
|
||||
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, metrics, time, metricTags);
|
||||
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, metrics, time);
|
||||
|
||||
long now = time.milliseconds();
|
||||
accum.append(tp1, key, value, null, maxBlockTimeMs);
|
||||
|
|
@ -256,7 +254,7 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testFlush() throws Exception {
|
||||
long lingerMs = Long.MAX_VALUE;
|
||||
final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time, metricTags);
|
||||
final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
|
||||
for (int i = 0; i < 100; i++)
|
||||
accum.append(new TopicPartition(topic, i % 3), key, value, null, maxBlockTimeMs);
|
||||
RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds());
|
||||
|
|
@ -280,7 +278,7 @@ public class RecordAccumulatorTest {
|
|||
public void testAbortIncompleteBatches() throws Exception {
|
||||
long lingerMs = Long.MAX_VALUE;
|
||||
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
|
||||
final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time, metricTags);
|
||||
final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
|
||||
class TestCallback implements Callback {
|
||||
@Override
|
||||
public void onCompletion(RecordMetadata metadata, Exception exception) {
|
||||
|
|
@ -303,7 +301,7 @@ public class RecordAccumulatorTest {
|
|||
public void testExpiredBatches() throws InterruptedException {
|
||||
Time time = new SystemTime();
|
||||
long now = time.milliseconds();
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time, metricTags);
|
||||
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time);
|
||||
int appends = 1024 / msgSize;
|
||||
for (int i = 0; i < appends; i++) {
|
||||
accum.append(tp1, key, value, null, maxBlockTimeMs);
|
||||
|
|
|
|||
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.common.Cluster;
|
|||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.metrics.KafkaMetric;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
|
@ -57,24 +58,29 @@ public class SenderTest {
|
|||
private int batchSize = 16 * 1024;
|
||||
private Metadata metadata = new Metadata(0, Long.MAX_VALUE);
|
||||
private Cluster cluster = TestUtils.singletonCluster("test", 1);
|
||||
private Metrics metrics = new Metrics(time);
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time, metricTags);
|
||||
private Sender sender = new Sender(client,
|
||||
metadata,
|
||||
this.accumulator,
|
||||
MAX_REQUEST_SIZE,
|
||||
ACKS_ALL,
|
||||
MAX_RETRIES,
|
||||
metrics,
|
||||
time,
|
||||
CLIENT_ID,
|
||||
REQUEST_TIMEOUT);
|
||||
private Metrics metrics = null;
|
||||
private RecordAccumulator accumulator = null;
|
||||
private Sender sender = null;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
metadata.update(cluster, time.milliseconds());
|
||||
Map<String, String> metricTags = new LinkedHashMap<String, String>();
|
||||
metricTags.put("client-id", CLIENT_ID);
|
||||
MetricConfig metricConfig = new MetricConfig().tags(metricTags);
|
||||
metrics = new Metrics(metricConfig, time);
|
||||
accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time);
|
||||
sender = new Sender(client,
|
||||
metadata,
|
||||
this.accumulator,
|
||||
MAX_REQUEST_SIZE,
|
||||
ACKS_ALL,
|
||||
MAX_RETRIES,
|
||||
metrics,
|
||||
time,
|
||||
CLIENT_ID,
|
||||
REQUEST_TIMEOUT);
|
||||
|
||||
metadata.update(cluster, time.milliseconds());
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
@ -110,8 +116,8 @@ public class SenderTest {
|
|||
sender.run(time.milliseconds());
|
||||
}
|
||||
Map<MetricName, KafkaMetric> allMetrics = metrics.metrics();
|
||||
KafkaMetric avgMetric = allMetrics.get(new MetricName("produce-throttle-time-avg", METRIC_GROUP, "", metricTags));
|
||||
KafkaMetric maxMetric = allMetrics.get(new MetricName("produce-throttle-time-max", METRIC_GROUP, "", metricTags));
|
||||
KafkaMetric avgMetric = allMetrics.get(metrics.metricName("produce-throttle-time-avg", METRIC_GROUP, ""));
|
||||
KafkaMetric maxMetric = allMetrics.get(metrics.metricName("produce-throttle-time-max", METRIC_GROUP, ""));
|
||||
assertEquals(200, avgMetric.value(), EPS);
|
||||
assertEquals(300, maxMetric.value(), EPS);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.common.metrics;
|
||||
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.metrics.stats.Avg;
|
||||
import org.apache.kafka.common.metrics.stats.Total;
|
||||
import org.junit.Test;
|
||||
|
|
@ -29,11 +28,11 @@ public class JmxReporterTest {
|
|||
try {
|
||||
metrics.addReporter(new JmxReporter());
|
||||
Sensor sensor = metrics.sensor("kafka.requests");
|
||||
sensor.add(new MetricName("pack.bean1.avg", "grp1"), new Avg());
|
||||
sensor.add(new MetricName("pack.bean2.total", "grp2"), new Total());
|
||||
sensor.add(metrics.metricName("pack.bean1.avg", "grp1"), new Avg());
|
||||
sensor.add(metrics.metricName("pack.bean2.total", "grp2"), new Total());
|
||||
Sensor sensor2 = metrics.sensor("kafka.blah");
|
||||
sensor2.add(new MetricName("pack.bean1.some", "grp1"), new Total());
|
||||
sensor2.add(new MetricName("pack.bean2.some", "grp1"), new Total());
|
||||
sensor2.add(metrics.metricName("pack.bean1.some", "grp1"), new Total());
|
||||
sensor2.add(metrics.metricName("pack.bean2.some", "grp1"), new Total());
|
||||
} finally {
|
||||
metrics.close();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -59,16 +59,16 @@ public class MetricsTest {
|
|||
|
||||
@Test
|
||||
public void testMetricName() {
|
||||
MetricName n1 = new MetricName("name", "group", "description", "key1", "value1", "key2", "value2");
|
||||
MetricName n1 = metrics.metricName("name", "group", "description", "key1", "value1", "key2", "value2");
|
||||
Map<String, String> tags = new HashMap<String, String>();
|
||||
tags.put("key1", "value1");
|
||||
tags.put("key2", "value2");
|
||||
MetricName n2 = new MetricName("name", "group", "description", tags);
|
||||
MetricName n2 = metrics.metricName("name", "group", "description", tags);
|
||||
assertEquals("metric names created in two different ways should be equal", n1, n2);
|
||||
|
||||
try {
|
||||
new MetricName("name", "group", "description", "key1");
|
||||
fail("Creating MetricName with an old number of keyValue should fail");
|
||||
metrics.metricName("name", "group", "description", "key1");
|
||||
fail("Creating MetricName with an odd number of keyValue should fail");
|
||||
} catch (IllegalArgumentException e) {
|
||||
// this is expected
|
||||
}
|
||||
|
|
@ -78,20 +78,20 @@ public class MetricsTest {
|
|||
public void testSimpleStats() throws Exception {
|
||||
ConstantMeasurable measurable = new ConstantMeasurable();
|
||||
|
||||
metrics.addMetric(new MetricName("direct.measurable", "grp1", "The fraction of time an appender waits for space allocation."), measurable);
|
||||
metrics.addMetric(metrics.metricName("direct.measurable", "grp1", "The fraction of time an appender waits for space allocation."), measurable);
|
||||
Sensor s = metrics.sensor("test.sensor");
|
||||
s.add(new MetricName("test.avg", "grp1"), new Avg());
|
||||
s.add(new MetricName("test.max", "grp1"), new Max());
|
||||
s.add(new MetricName("test.min", "grp1"), new Min());
|
||||
s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS));
|
||||
s.add(new MetricName("test.occurences", "grp1"), new Rate(TimeUnit.SECONDS, new Count()));
|
||||
s.add(new MetricName("test.count", "grp1"), new Count());
|
||||
s.add(metrics.metricName("test.avg", "grp1"), new Avg());
|
||||
s.add(metrics.metricName("test.max", "grp1"), new Max());
|
||||
s.add(metrics.metricName("test.min", "grp1"), new Min());
|
||||
s.add(metrics.metricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS));
|
||||
s.add(metrics.metricName("test.occurences", "grp1"), new Rate(TimeUnit.SECONDS, new Count()));
|
||||
s.add(metrics.metricName("test.count", "grp1"), new Count());
|
||||
s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT,
|
||||
new Percentile(new MetricName("test.median", "grp1"), 50.0),
|
||||
new Percentile(new MetricName("test.perc99_9", "grp1"), 99.9)));
|
||||
new Percentile(metrics.metricName("test.median", "grp1"), 50.0),
|
||||
new Percentile(metrics.metricName("test.perc99_9", "grp1"), 99.9)));
|
||||
|
||||
Sensor s2 = metrics.sensor("test.sensor2");
|
||||
s2.add(new MetricName("s2.total", "grp1"), new Total());
|
||||
s2.add(metrics.metricName("s2.total", "grp1"), new Total());
|
||||
s2.record(5.0);
|
||||
|
||||
int sum = 0;
|
||||
|
|
@ -103,38 +103,38 @@ public class MetricsTest {
|
|||
// prior to any time passing
|
||||
double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0;
|
||||
assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), count / elapsedSecs,
|
||||
metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS);
|
||||
metrics.metrics().get(metrics.metricName("test.occurences", "grp1")).value(), EPS);
|
||||
|
||||
// pretend 2 seconds passed...
|
||||
long sleepTimeMs = 2;
|
||||
time.sleep(sleepTimeMs * 1000);
|
||||
elapsedSecs += sleepTimeMs;
|
||||
|
||||
assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS);
|
||||
assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS);
|
||||
assertEquals("Max(0...9) = 9", count - 1, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS);
|
||||
assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS);
|
||||
assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(metrics.metricName("s2.total", "grp1")).value(), EPS);
|
||||
assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(metrics.metricName("test.avg", "grp1")).value(), EPS);
|
||||
assertEquals("Max(0...9) = 9", count - 1, metrics.metrics().get(metrics.metricName("test.max", "grp1")).value(), EPS);
|
||||
assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(metrics.metricName("test.min", "grp1")).value(), EPS);
|
||||
assertEquals("Rate(0...9) = 1.40625",
|
||||
sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS);
|
||||
sum / elapsedSecs, metrics.metrics().get(metrics.metricName("test.rate", "grp1")).value(), EPS);
|
||||
assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs),
|
||||
count / elapsedSecs,
|
||||
metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS);
|
||||
metrics.metrics().get(metrics.metricName("test.occurences", "grp1")).value(), EPS);
|
||||
assertEquals("Count(0...9) = 10",
|
||||
(double) count, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS);
|
||||
(double) count, metrics.metrics().get(metrics.metricName("test.count", "grp1")).value(), EPS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHierarchicalSensors() {
|
||||
Sensor parent1 = metrics.sensor("test.parent1");
|
||||
parent1.add(new MetricName("test.parent1.count", "grp1"), new Count());
|
||||
parent1.add(metrics.metricName("test.parent1.count", "grp1"), new Count());
|
||||
Sensor parent2 = metrics.sensor("test.parent2");
|
||||
parent2.add(new MetricName("test.parent2.count", "grp1"), new Count());
|
||||
parent2.add(metrics.metricName("test.parent2.count", "grp1"), new Count());
|
||||
Sensor child1 = metrics.sensor("test.child1", parent1, parent2);
|
||||
child1.add(new MetricName("test.child1.count", "grp1"), new Count());
|
||||
child1.add(metrics.metricName("test.child1.count", "grp1"), new Count());
|
||||
Sensor child2 = metrics.sensor("test.child2", parent1);
|
||||
child2.add(new MetricName("test.child2.count", "grp1"), new Count());
|
||||
child2.add(metrics.metricName("test.child2.count", "grp1"), new Count());
|
||||
Sensor grandchild = metrics.sensor("test.grandchild", child1);
|
||||
grandchild.add(new MetricName("test.grandchild.count", "grp1"), new Count());
|
||||
grandchild.add(metrics.metricName("test.grandchild.count", "grp1"), new Count());
|
||||
|
||||
/* increment each sensor one time */
|
||||
parent1.record();
|
||||
|
|
@ -167,75 +167,76 @@ public class MetricsTest {
|
|||
|
||||
@Test
|
||||
public void testRemoveSensor() {
|
||||
int size = metrics.metrics().size();
|
||||
Sensor parent1 = metrics.sensor("test.parent1");
|
||||
parent1.add(new MetricName("test.parent1.count", "grp1"), new Count());
|
||||
parent1.add(metrics.metricName("test.parent1.count", "grp1"), new Count());
|
||||
Sensor parent2 = metrics.sensor("test.parent2");
|
||||
parent2.add(new MetricName("test.parent2.count", "grp1"), new Count());
|
||||
parent2.add(metrics.metricName("test.parent2.count", "grp1"), new Count());
|
||||
Sensor child1 = metrics.sensor("test.child1", parent1, parent2);
|
||||
child1.add(new MetricName("test.child1.count", "grp1"), new Count());
|
||||
child1.add(metrics.metricName("test.child1.count", "grp1"), new Count());
|
||||
Sensor child2 = metrics.sensor("test.child2", parent2);
|
||||
child2.add(new MetricName("test.child2.count", "grp1"), new Count());
|
||||
child2.add(metrics.metricName("test.child2.count", "grp1"), new Count());
|
||||
Sensor grandChild1 = metrics.sensor("test.gchild2", child2);
|
||||
grandChild1.add(new MetricName("test.gchild2.count", "grp1"), new Count());
|
||||
grandChild1.add(metrics.metricName("test.gchild2.count", "grp1"), new Count());
|
||||
|
||||
Sensor sensor = metrics.getSensor("test.parent1");
|
||||
assertNotNull(sensor);
|
||||
metrics.removeSensor("test.parent1");
|
||||
assertNull(metrics.getSensor("test.parent1"));
|
||||
assertNull(metrics.metrics().get(new MetricName("test.parent1.count", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test.parent1.count", "grp1")));
|
||||
assertNull(metrics.getSensor("test.child1"));
|
||||
assertNull(metrics.childrenSensors().get(sensor));
|
||||
assertNull(metrics.metrics().get(new MetricName("test.child1.count", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test.child1.count", "grp1")));
|
||||
|
||||
sensor = metrics.getSensor("test.gchild2");
|
||||
assertNotNull(sensor);
|
||||
metrics.removeSensor("test.gchild2");
|
||||
assertNull(metrics.getSensor("test.gchild2"));
|
||||
assertNull(metrics.childrenSensors().get(sensor));
|
||||
assertNull(metrics.metrics().get(new MetricName("test.gchild2.count", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test.gchild2.count", "grp1")));
|
||||
|
||||
sensor = metrics.getSensor("test.child2");
|
||||
assertNotNull(sensor);
|
||||
metrics.removeSensor("test.child2");
|
||||
assertNull(metrics.getSensor("test.child2"));
|
||||
assertNull(metrics.childrenSensors().get(sensor));
|
||||
assertNull(metrics.metrics().get(new MetricName("test.child2.count", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test.child2.count", "grp1")));
|
||||
|
||||
sensor = metrics.getSensor("test.parent2");
|
||||
assertNotNull(sensor);
|
||||
metrics.removeSensor("test.parent2");
|
||||
assertNull(metrics.getSensor("test.parent2"));
|
||||
assertNull(metrics.childrenSensors().get(sensor));
|
||||
assertNull(metrics.metrics().get(new MetricName("test.parent2.count", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test.parent2.count", "grp1")));
|
||||
|
||||
assertEquals(0, metrics.metrics().size());
|
||||
assertEquals(size, metrics.metrics().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveInactiveMetrics() {
|
||||
Sensor s1 = metrics.sensor("test.s1", null, 1);
|
||||
s1.add(new MetricName("test.s1.count", "grp1"), new Count());
|
||||
s1.add(metrics.metricName("test.s1.count", "grp1"), new Count());
|
||||
|
||||
Sensor s2 = metrics.sensor("test.s2", null, 3);
|
||||
s2.add(new MetricName("test.s2.count", "grp1"), new Count());
|
||||
s2.add(metrics.metricName("test.s2.count", "grp1"), new Count());
|
||||
|
||||
Metrics.ExpireSensorTask purger = metrics.new ExpireSensorTask();
|
||||
purger.run();
|
||||
assertNotNull("Sensor test.s1 must be present", metrics.getSensor("test.s1"));
|
||||
assertNotNull("MetricName test.s1.count must be present",
|
||||
metrics.metrics().get(new MetricName("test.s1.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s1.count", "grp1")));
|
||||
assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2"));
|
||||
assertNotNull("MetricName test.s2.count must be present",
|
||||
metrics.metrics().get(new MetricName("test.s2.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s2.count", "grp1")));
|
||||
|
||||
time.sleep(1001);
|
||||
purger.run();
|
||||
assertNull("Sensor test.s1 should have been purged", metrics.getSensor("test.s1"));
|
||||
assertNull("MetricName test.s1.count should have been purged",
|
||||
metrics.metrics().get(new MetricName("test.s1.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s1.count", "grp1")));
|
||||
assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2"));
|
||||
assertNotNull("MetricName test.s2.count must be present",
|
||||
metrics.metrics().get(new MetricName("test.s2.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s2.count", "grp1")));
|
||||
|
||||
// record a value in sensor s2. This should reset the clock for that sensor.
|
||||
// It should not get purged at the 3 second mark after creation
|
||||
|
|
@ -244,36 +245,37 @@ public class MetricsTest {
|
|||
purger.run();
|
||||
assertNotNull("Sensor test.s2 must be present", metrics.getSensor("test.s2"));
|
||||
assertNotNull("MetricName test.s2.count must be present",
|
||||
metrics.metrics().get(new MetricName("test.s2.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s2.count", "grp1")));
|
||||
|
||||
// After another 1 second sleep, the metric should be purged
|
||||
time.sleep(1000);
|
||||
purger.run();
|
||||
assertNull("Sensor test.s2 should have been purged", metrics.getSensor("test.s1"));
|
||||
assertNull("MetricName test.s2.count should have been purged",
|
||||
metrics.metrics().get(new MetricName("test.s1.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s1.count", "grp1")));
|
||||
|
||||
// After purging, it should be possible to recreate a metric
|
||||
s1 = metrics.sensor("test.s1", null, 1);
|
||||
s1.add(new MetricName("test.s1.count", "grp1"), new Count());
|
||||
s1.add(metrics.metricName("test.s1.count", "grp1"), new Count());
|
||||
assertNotNull("Sensor test.s1 must be present", metrics.getSensor("test.s1"));
|
||||
assertNotNull("MetricName test.s1.count must be present",
|
||||
metrics.metrics().get(new MetricName("test.s1.count", "grp1")));
|
||||
metrics.metrics().get(metrics.metricName("test.s1.count", "grp1")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveMetric() {
|
||||
metrics.addMetric(new MetricName("test1", "grp1"), new Count());
|
||||
metrics.addMetric(new MetricName("test2", "grp1"), new Count());
|
||||
int size = metrics.metrics().size();
|
||||
metrics.addMetric(metrics.metricName("test1", "grp1"), new Count());
|
||||
metrics.addMetric(metrics.metricName("test2", "grp1"), new Count());
|
||||
|
||||
assertNotNull(metrics.removeMetric(new MetricName("test1", "grp1")));
|
||||
assertNull(metrics.metrics().get(new MetricName("test1", "grp1")));
|
||||
assertNotNull(metrics.metrics().get(new MetricName("test2", "grp1")));
|
||||
assertNotNull(metrics.removeMetric(metrics.metricName("test1", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test1", "grp1")));
|
||||
assertNotNull(metrics.metrics().get(metrics.metricName("test2", "grp1")));
|
||||
|
||||
assertNotNull(metrics.removeMetric(new MetricName("test2", "grp1")));
|
||||
assertNull(metrics.metrics().get(new MetricName("test2", "grp1")));
|
||||
assertNotNull(metrics.removeMetric(metrics.metricName("test2", "grp1")));
|
||||
assertNull(metrics.metrics().get(metrics.metricName("test2", "grp1")));
|
||||
|
||||
assertEquals(0, metrics.metrics().size());
|
||||
assertEquals(size, metrics.metrics().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -313,15 +315,15 @@ public class MetricsTest {
|
|||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testDuplicateMetricName() {
|
||||
metrics.sensor("test").add(new MetricName("test", "grp1"), new Avg());
|
||||
metrics.sensor("test2").add(new MetricName("test", "grp1"), new Total());
|
||||
metrics.sensor("test").add(metrics.metricName("test", "grp1"), new Avg());
|
||||
metrics.sensor("test2").add(metrics.metricName("test", "grp1"), new Total());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQuotas() {
|
||||
Sensor sensor = metrics.sensor("test");
|
||||
sensor.add(new MetricName("test1.total", "grp1"), new Total(), new MetricConfig().quota(Quota.upperBound(5.0)));
|
||||
sensor.add(new MetricName("test2.total", "grp1"), new Total(), new MetricConfig().quota(Quota.lowerBound(0.0)));
|
||||
sensor.add(metrics.metricName("test1.total", "grp1"), new Total(), new MetricConfig().quota(Quota.upperBound(5.0)));
|
||||
sensor.add(metrics.metricName("test2.total", "grp1"), new Total(), new MetricConfig().quota(Quota.lowerBound(0.0)));
|
||||
sensor.record(5.0);
|
||||
try {
|
||||
sensor.record(1.0);
|
||||
|
|
@ -329,7 +331,7 @@ public class MetricsTest {
|
|||
} catch (QuotaViolationException e) {
|
||||
// this is good
|
||||
}
|
||||
assertEquals(6.0, metrics.metrics().get(new MetricName("test1.total", "grp1")).value(), EPS);
|
||||
assertEquals(6.0, metrics.metrics().get(metrics.metricName("test1.total", "grp1")).value(), EPS);
|
||||
sensor.record(-6.0);
|
||||
try {
|
||||
sensor.record(-1.0);
|
||||
|
|
@ -358,15 +360,15 @@ public class MetricsTest {
|
|||
0.0,
|
||||
100.0,
|
||||
BucketSizing.CONSTANT,
|
||||
new Percentile(new MetricName("test.p25", "grp1"), 25),
|
||||
new Percentile(new MetricName("test.p50", "grp1"), 50),
|
||||
new Percentile(new MetricName("test.p75", "grp1"), 75));
|
||||
new Percentile(metrics.metricName("test.p25", "grp1"), 25),
|
||||
new Percentile(metrics.metricName("test.p50", "grp1"), 50),
|
||||
new Percentile(metrics.metricName("test.p75", "grp1"), 75));
|
||||
MetricConfig config = new MetricConfig().eventWindow(50).samples(2);
|
||||
Sensor sensor = metrics.sensor("test", config);
|
||||
sensor.add(percs);
|
||||
Metric p25 = this.metrics.metrics().get(new MetricName("test.p25", "grp1"));
|
||||
Metric p50 = this.metrics.metrics().get(new MetricName("test.p50", "grp1"));
|
||||
Metric p75 = this.metrics.metrics().get(new MetricName("test.p75", "grp1"));
|
||||
Metric p25 = this.metrics.metrics().get(metrics.metricName("test.p25", "grp1"));
|
||||
Metric p50 = this.metrics.metrics().get(metrics.metricName("test.p50", "grp1"));
|
||||
Metric p75 = this.metrics.metrics().get(metrics.metricName("test.p75", "grp1"));
|
||||
|
||||
// record two windows worth of sequential values
|
||||
for (int i = 0; i < buckets; i++)
|
||||
|
|
@ -389,7 +391,7 @@ public class MetricsTest {
|
|||
// Use the default time window. Set 3 samples
|
||||
MetricConfig cfg = new MetricConfig().samples(3);
|
||||
Sensor s = metrics.sensor("test.sensor", cfg);
|
||||
s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS));
|
||||
s.add(metrics.metricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS));
|
||||
|
||||
int sum = 0;
|
||||
int count = cfg.samples() - 1;
|
||||
|
|
@ -406,7 +408,7 @@ public class MetricsTest {
|
|||
// prior to any time passing
|
||||
double elapsedSecs = (cfg.timeWindowMs() * (cfg.samples() - 1) + cfg.timeWindowMs() / 2) / 1000.0;
|
||||
|
||||
KafkaMetric km = metrics.metrics().get(new MetricName("test.rate", "grp1"));
|
||||
KafkaMetric km = metrics.metrics().get(metrics.metricName("test.rate", "grp1"));
|
||||
assertEquals("Rate(0...2) = 2.666", sum / elapsedSecs, km.value(), EPS);
|
||||
assertEquals("Elapsed Time = 75 seconds", elapsedSecs,
|
||||
((Rate) km.measurable()).windowSize(cfg, time.milliseconds()) / 1000, EPS);
|
||||
|
|
|
|||
|
|
@ -54,7 +54,7 @@ public class SelectorTest {
|
|||
this.channelBuilder = new PlaintextChannelBuilder();
|
||||
this.channelBuilder.configure(configs);
|
||||
this.metrics = new Metrics();
|
||||
this.selector = new Selector(5000, this.metrics, time, "MetricGroup", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
this.selector = new Selector(5000, this.metrics, time, "MetricGroup", channelBuilder);
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
|||
|
|
@ -18,7 +18,6 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.io.File;
|
||||
|
|
@ -56,7 +55,7 @@ public class SslSelectorTest extends SelectorTest {
|
|||
this.channelBuilder = new SslChannelBuilder(Mode.CLIENT);
|
||||
this.channelBuilder.configure(sslClientConfigs);
|
||||
this.metrics = new Metrics();
|
||||
this.selector = new Selector(5000, metrics, time, "MetricGroup", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
this.selector = new Selector(5000, metrics, time, "MetricGroup", channelBuilder);
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
@ -84,7 +83,7 @@ public class SslSelectorTest extends SelectorTest {
|
|||
}
|
||||
};
|
||||
channelBuilder.configure(sslClientConfigs);
|
||||
Selector selector = new Selector(5000, metrics, time, "MetricGroup2", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
Selector selector = new Selector(5000, metrics, time, "MetricGroup2", channelBuilder);
|
||||
try {
|
||||
int reqs = 500;
|
||||
String node = "0";
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
|
@ -75,7 +74,7 @@ public class SslTransportLayerTest {
|
|||
|
||||
this.channelBuilder = new SslChannelBuilder(Mode.CLIENT);
|
||||
this.channelBuilder.configure(sslClientConfigs);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
@ -452,7 +451,7 @@ public class SslTransportLayerTest {
|
|||
|
||||
};
|
||||
this.channelBuilder.configure(sslClientConfigs);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
|
||||
}
|
||||
|
||||
private static class CertStores {
|
||||
|
|
@ -560,7 +559,7 @@ public class SslTransportLayerTest {
|
|||
this.newChannels = Collections.synchronizedList(new ArrayList<SocketChannel>());
|
||||
SslChannelBuilder channelBuilder = new SslChannelBuilder(Mode.SERVER);
|
||||
channelBuilder.configure(sslServerConfigs);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap<String, String>(), channelBuilder);
|
||||
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder);
|
||||
setName("echoserver");
|
||||
setDaemon(true);
|
||||
acceptorThread = new AcceptorThread();
|
||||
|
|
|
|||
|
|
@ -14,7 +14,6 @@ package org.apache.kafka.test;
|
|||
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.metrics.Sensor;
|
||||
import org.apache.kafka.common.metrics.stats.Avg;
|
||||
|
|
@ -33,15 +32,15 @@ public class MetricsBench {
|
|||
Sensor parent = metrics.sensor("parent");
|
||||
Sensor child = metrics.sensor("child", parent);
|
||||
for (Sensor sensor : Arrays.asList(parent, child)) {
|
||||
sensor.add(new MetricName(sensor.name() + ".avg", "grp1"), new Avg());
|
||||
sensor.add(new MetricName(sensor.name() + ".count", "grp1"), new Count());
|
||||
sensor.add(new MetricName(sensor.name() + ".max", "grp1"), new Max());
|
||||
sensor.add(metrics.metricName(sensor.name() + ".avg", "grp1"), new Avg());
|
||||
sensor.add(metrics.metricName(sensor.name() + ".count", "grp1"), new Count());
|
||||
sensor.add(metrics.metricName(sensor.name() + ".max", "grp1"), new Max());
|
||||
sensor.add(new Percentiles(1024,
|
||||
0.0,
|
||||
iters,
|
||||
BucketSizing.CONSTANT,
|
||||
new Percentile(new MetricName(sensor.name() + ".median", "grp1"), 50.0),
|
||||
new Percentile(new MetricName(sensor.name() + ".p_99", "grp1"), 99.0)));
|
||||
new Percentile(metrics.metricName(sensor.name() + ".median", "grp1"), 50.0),
|
||||
new Percentile(metrics.metricName(sensor.name() + ".p_99", "grp1"), 99.0)));
|
||||
}
|
||||
long start = System.nanoTime();
|
||||
for (int i = 0; i < iters; i++)
|
||||
|
|
|
|||
|
|
@ -18,7 +18,6 @@ package org.apache.kafka.connect.runtime.distributed;
|
|||
|
||||
import org.apache.kafka.clients.consumer.internals.AbstractCoordinator;
|
||||
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.metrics.Measurable;
|
||||
import org.apache.kafka.common.metrics.MetricConfig;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
|
|
@ -67,7 +66,6 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos
|
|||
int heartbeatIntervalMs,
|
||||
Metrics metrics,
|
||||
String metricGrpPrefix,
|
||||
Map<String, String> metricTags,
|
||||
Time time,
|
||||
long retryBackoffMs,
|
||||
String restUrl,
|
||||
|
|
@ -79,13 +77,12 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos
|
|||
heartbeatIntervalMs,
|
||||
metrics,
|
||||
metricGrpPrefix,
|
||||
metricTags,
|
||||
time,
|
||||
retryBackoffMs);
|
||||
this.restUrl = restUrl;
|
||||
this.configStorage = configStorage;
|
||||
this.assignmentSnapshot = null;
|
||||
this.sensors = new WorkerCoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
|
||||
this.sensors = new WorkerCoordinatorMetrics(metrics, metricGrpPrefix);
|
||||
this.listener = listener;
|
||||
this.rejoinRequested = false;
|
||||
}
|
||||
|
|
@ -254,7 +251,7 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos
|
|||
public final Metrics metrics;
|
||||
public final String metricGrpName;
|
||||
|
||||
public WorkerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
|
||||
public WorkerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) {
|
||||
this.metrics = metrics;
|
||||
this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
|
||||
|
||||
|
|
@ -270,16 +267,12 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos
|
|||
}
|
||||
};
|
||||
|
||||
metrics.addMetric(new MetricName("assigned-connectors",
|
||||
this.metricGrpName,
|
||||
"The number of connector instances currently assigned to this consumer",
|
||||
tags),
|
||||
numConnectors);
|
||||
metrics.addMetric(new MetricName("assigned-tasks",
|
||||
this.metricGrpName,
|
||||
"The number of tasks currently assigned to this consumer",
|
||||
tags),
|
||||
numTasks);
|
||||
metrics.addMetric(metrics.metricName("assigned-connectors",
|
||||
this.metricGrpName,
|
||||
"The number of connector instances currently assigned to this consumer"), numConnectors);
|
||||
metrics.addMetric(metrics.metricName("assigned-tasks",
|
||||
this.metricGrpName,
|
||||
"The number of tasks currently assigned to this consumer"), numTasks);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -71,10 +71,13 @@ public class WorkerGroupMember {
|
|||
try {
|
||||
this.time = new SystemTime();
|
||||
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS);
|
||||
String clientIdConfig = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
|
||||
clientId = clientIdConfig.length() <= 0 ? "connect-" + CONNECT_CLIENT_ID_SEQUENCE.getAndIncrement() : clientIdConfig;
|
||||
Map<String, String> metricsTags = new LinkedHashMap<>();
|
||||
metricsTags.put("client-id", clientId);
|
||||
MetricConfig metricConfig = new MetricConfig().samples(config.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG))
|
||||
.timeWindow(config.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS)
|
||||
.tags(metricsTags);
|
||||
List<MetricsReporter> reporters = config.getConfiguredInstances(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class);
|
||||
reporters.add(new JmxReporter(JMX_PREFIX));
|
||||
this.metrics = new Metrics(metricConfig, reporters, time);
|
||||
|
|
@ -83,11 +86,9 @@ public class WorkerGroupMember {
|
|||
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG));
|
||||
this.metadata.update(Cluster.bootstrap(addresses), 0);
|
||||
String metricGrpPrefix = "connect";
|
||||
Map<String, String> metricsTags = new LinkedHashMap<>();
|
||||
metricsTags.put("client-id", clientId);
|
||||
ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values());
|
||||
NetworkClient netClient = new NetworkClient(
|
||||
new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, metricsTags, channelBuilder),
|
||||
new Selector(config.getLong(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, metricGrpPrefix, channelBuilder),
|
||||
this.metadata,
|
||||
clientId,
|
||||
100, // a fixed large enough value will suffice
|
||||
|
|
@ -102,7 +103,6 @@ public class WorkerGroupMember {
|
|||
config.getInt(DistributedConfig.HEARTBEAT_INTERVAL_MS_CONFIG),
|
||||
metrics,
|
||||
metricGrpPrefix,
|
||||
metricsTags,
|
||||
this.time,
|
||||
retryBackoffMs,
|
||||
restUrl,
|
||||
|
|
|
|||
|
|
@ -47,7 +47,6 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
|
@ -75,7 +74,6 @@ public class WorkerCoordinatorTest {
|
|||
private Node node = cluster.nodes().get(0);
|
||||
private Metadata metadata;
|
||||
private Metrics metrics;
|
||||
private Map<String, String> metricTags = new LinkedHashMap<>();
|
||||
private ConsumerNetworkClient consumerClient;
|
||||
private MockRebalanceListener rebalanceListener;
|
||||
@Mock private KafkaConfigStorage configStorage;
|
||||
|
|
@ -103,7 +101,6 @@ public class WorkerCoordinatorTest {
|
|||
heartbeatIntervalMs,
|
||||
metrics,
|
||||
"consumer" + groupId,
|
||||
metricTags,
|
||||
time,
|
||||
retryBackoffMs,
|
||||
LEADER_URL,
|
||||
|
|
|
|||
|
|
@ -226,7 +226,6 @@ object AdminClient {
|
|||
metrics,
|
||||
time,
|
||||
"admin",
|
||||
Map[String, String](),
|
||||
channelBuilder)
|
||||
|
||||
val networkClient = new NetworkClient(
|
||||
|
|
|
|||
|
|
@ -111,6 +111,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
|||
private val zkCommitMeter = newMeter("ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId))
|
||||
private val rebalanceTimer = new KafkaTimer(newTimer("RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, Map("clientId" -> config.clientId)))
|
||||
|
||||
newGauge(
|
||||
"yammer-metrics-count",
|
||||
new Gauge[Int] {
|
||||
def value = {
|
||||
com.yammer.metrics.Metrics.defaultRegistry().allMetrics().size()
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
val consumerIdString = {
|
||||
var consumerUuid : String = null
|
||||
config.consumerId match {
|
||||
|
|
|
|||
|
|
@ -70,7 +70,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time
|
|||
private val allMetricNames = (0 until totalProcessorThreads).map { i =>
|
||||
val tags = new util.HashMap[String, String]()
|
||||
tags.put("networkProcessor", i.toString)
|
||||
new MetricName("io-wait-ratio", "socket-server-metrics", tags)
|
||||
metrics.metricName("io-wait-ratio", "socket-server-metrics", tags)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -384,7 +384,7 @@ private[kafka] class Processor(val id: Int,
|
|||
newGauge("IdlePercent",
|
||||
new Gauge[Double] {
|
||||
def value = {
|
||||
metrics.metrics().get(new MetricName("io-wait-ratio", "socket-server-metrics", metricTags)).value()
|
||||
metrics.metrics().get(metrics.metricName("io-wait-ratio", "socket-server-metrics", metricTags)).value()
|
||||
}
|
||||
},
|
||||
metricTags.asScala
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
throttledRequestReaper.start()
|
||||
|
||||
private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue")
|
||||
delayQueueSensor.add(new MetricName("queue-size",
|
||||
delayQueueSensor.add(metrics.metricName("queue-size",
|
||||
apiKey,
|
||||
"Tracks the size of the delay queue"), new Total())
|
||||
|
||||
|
|
@ -206,7 +206,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
throttleTimeSensor = metrics.sensor(throttleTimeSensorName,
|
||||
null,
|
||||
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds)
|
||||
throttleTimeSensor.add(new MetricName("throttle-time",
|
||||
throttleTimeSensor.add(metrics.metricName("throttle-time",
|
||||
apiKey,
|
||||
"Tracking average throttle-time per client",
|
||||
"client-id",
|
||||
|
|
@ -271,7 +271,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
}
|
||||
|
||||
private def clientRateMetricName(clientId: String): MetricName = {
|
||||
new MetricName("byte-rate", apiKey,
|
||||
metrics.metricName("byte-rate", apiKey,
|
||||
"Tracking byte-rate per client",
|
||||
"client-id", clientId)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -141,6 +141,15 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
|
|||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"yammer-metrics-count",
|
||||
new Gauge[Int] {
|
||||
def value = {
|
||||
com.yammer.metrics.Metrics.defaultRegistry().allMetrics().size()
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
/**
|
||||
* Start up API for bringing up a single instance of the Kafka server.
|
||||
* Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers
|
||||
|
|
|
|||
|
|
@ -90,9 +90,6 @@ object JmxTool extends Logging {
|
|||
List(null)
|
||||
|
||||
val names = queries.map((name: ObjectName) => mbsc.queryNames(name, null): mutable.Set[ObjectName]).flatten
|
||||
val allAttributes: Iterable[(ObjectName, Array[String])] =
|
||||
names.map((name: ObjectName) => (name, mbsc.getMBeanInfo(name).getAttributes().map(_.getName)))
|
||||
|
||||
|
||||
val numExpectedAttributes: Map[ObjectName, Int] =
|
||||
attributesWhitelistExists match {
|
||||
|
|
|
|||
|
|
@ -125,10 +125,10 @@ class QuotasTest extends KafkaServerTestHarness {
|
|||
val numRecords = 1000
|
||||
produce(producers.head, numRecords)
|
||||
|
||||
val producerMetricName = new MetricName("throttle-time",
|
||||
ApiKeys.PRODUCE.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", producerId1)
|
||||
val producerMetricName = leaderNode.metrics.metricName("throttle-time",
|
||||
ApiKeys.PRODUCE.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", producerId1)
|
||||
assertTrue("Should have been throttled", allMetrics(producerMetricName).value() > 0)
|
||||
|
||||
// Consumer should read in a bursty manner and get throttled immediately
|
||||
|
|
@ -136,10 +136,10 @@ class QuotasTest extends KafkaServerTestHarness {
|
|||
// The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately
|
||||
val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build()
|
||||
replicaConsumers.head.fetch(request)
|
||||
val consumerMetricName = new MetricName("throttle-time",
|
||||
ApiKeys.FETCH.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", consumerId1)
|
||||
val consumerMetricName = leaderNode.metrics.metricName("throttle-time",
|
||||
ApiKeys.FETCH.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", consumerId1)
|
||||
assertTrue("Should have been throttled", allMetrics(consumerMetricName).value() > 0)
|
||||
}
|
||||
|
||||
|
|
@ -166,10 +166,10 @@ class QuotasTest extends KafkaServerTestHarness {
|
|||
val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala
|
||||
val numRecords = 1000
|
||||
produce(producers(1), numRecords)
|
||||
val producerMetricName = new MetricName("throttle-time",
|
||||
ApiKeys.PRODUCE.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", producerId2)
|
||||
val producerMetricName = leaderNode.metrics.metricName("throttle-time",
|
||||
ApiKeys.PRODUCE.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", producerId2)
|
||||
assertEquals("Should not have been throttled", 0.0, allMetrics(producerMetricName).value(), 0.0)
|
||||
|
||||
// The "client" consumer does not get throttled.
|
||||
|
|
@ -177,10 +177,10 @@ class QuotasTest extends KafkaServerTestHarness {
|
|||
// The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately
|
||||
val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build()
|
||||
replicaConsumers(1).fetch(request)
|
||||
val consumerMetricName = new MetricName("throttle-time",
|
||||
ApiKeys.FETCH.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", consumerId2)
|
||||
val consumerMetricName = leaderNode.metrics.metricName("throttle-time",
|
||||
ApiKeys.FETCH.name,
|
||||
"Tracking throttle-time per client",
|
||||
"client-id", consumerId2)
|
||||
assertEquals("Should not have been throttled", 0.0, allMetrics(consumerMetricName).value(), 0.0)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -79,7 +79,7 @@ class ClientQuotaManagerTest {
|
|||
def testQuotaViolation() {
|
||||
val metrics = newMetrics
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, "producer", time)
|
||||
val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", ""))
|
||||
val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "producer", ""))
|
||||
try {
|
||||
/* We have 10 second windows. Make sure that there is no quota violation
|
||||
* if we produce under the quota
|
||||
|
|
|
|||
|
|
@ -680,30 +680,30 @@ public class StreamThread extends Thread {
|
|||
this.metricTags.put("client-id", clientId + "-" + getName());
|
||||
|
||||
this.commitTimeSensor = metrics.sensor("commit-time");
|
||||
this.commitTimeSensor.add(new MetricName("commit-time-avg", metricGrpName, "The average commit time in ms", metricTags), new Avg());
|
||||
this.commitTimeSensor.add(new MetricName("commit-time-max", metricGrpName, "The maximum commit time in ms", metricTags), new Max());
|
||||
this.commitTimeSensor.add(new MetricName("commit-calls-rate", metricGrpName, "The average per-second number of commit calls", metricTags), new Rate(new Count()));
|
||||
this.commitTimeSensor.add(metrics.metricName("commit-time-avg", metricGrpName, "The average commit time in ms", metricTags), new Avg());
|
||||
this.commitTimeSensor.add(metrics.metricName("commit-time-max", metricGrpName, "The maximum commit time in ms", metricTags), new Max());
|
||||
this.commitTimeSensor.add(metrics.metricName("commit-calls-rate", metricGrpName, "The average per-second number of commit calls", metricTags), new Rate(new Count()));
|
||||
|
||||
this.pollTimeSensor = metrics.sensor("poll-time");
|
||||
this.pollTimeSensor.add(new MetricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg());
|
||||
this.pollTimeSensor.add(new MetricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max());
|
||||
this.pollTimeSensor.add(new MetricName("poll-calls-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count()));
|
||||
this.pollTimeSensor.add(metrics.metricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg());
|
||||
this.pollTimeSensor.add(metrics.metricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max());
|
||||
this.pollTimeSensor.add(metrics.metricName("poll-calls-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count()));
|
||||
|
||||
this.processTimeSensor = metrics.sensor("process-time");
|
||||
this.processTimeSensor.add(new MetricName("process-time-avg-ms", metricGrpName, "The average process time in ms", metricTags), new Avg());
|
||||
this.processTimeSensor.add(new MetricName("process-time-max-ms", metricGrpName, "The maximum process time in ms", metricTags), new Max());
|
||||
this.processTimeSensor.add(new MetricName("process-calls-rate", metricGrpName, "The average per-second number of process calls", metricTags), new Rate(new Count()));
|
||||
this.processTimeSensor.add(metrics.metricName("process-time-avg-ms", metricGrpName, "The average process time in ms", metricTags), new Avg());
|
||||
this.processTimeSensor.add(metrics.metricName("process-time-max-ms", metricGrpName, "The maximum process time in ms", metricTags), new Max());
|
||||
this.processTimeSensor.add(metrics.metricName("process-calls-rate", metricGrpName, "The average per-second number of process calls", metricTags), new Rate(new Count()));
|
||||
|
||||
this.punctuateTimeSensor = metrics.sensor("punctuate-time");
|
||||
this.punctuateTimeSensor.add(new MetricName("punctuate-time-avg", metricGrpName, "The average punctuate time in ms", metricTags), new Avg());
|
||||
this.punctuateTimeSensor.add(new MetricName("punctuate-time-max", metricGrpName, "The maximum punctuate time in ms", metricTags), new Max());
|
||||
this.punctuateTimeSensor.add(new MetricName("punctuate-calls-rate", metricGrpName, "The average per-second number of punctuate calls", metricTags), new Rate(new Count()));
|
||||
this.punctuateTimeSensor.add(metrics.metricName("punctuate-time-avg", metricGrpName, "The average punctuate time in ms", metricTags), new Avg());
|
||||
this.punctuateTimeSensor.add(metrics.metricName("punctuate-time-max", metricGrpName, "The maximum punctuate time in ms", metricTags), new Max());
|
||||
this.punctuateTimeSensor.add(metrics.metricName("punctuate-calls-rate", metricGrpName, "The average per-second number of punctuate calls", metricTags), new Rate(new Count()));
|
||||
|
||||
this.taskCreationSensor = metrics.sensor("task-creation");
|
||||
this.taskCreationSensor.add(new MetricName("task-creation-rate", metricGrpName, "The average per-second number of newly created tasks", metricTags), new Rate(new Count()));
|
||||
this.taskCreationSensor.add(metrics.metricName("task-creation-rate", metricGrpName, "The average per-second number of newly created tasks", metricTags), new Rate(new Count()));
|
||||
|
||||
this.taskDestructionSensor = metrics.sensor("task-destruction");
|
||||
this.taskDestructionSensor.add(new MetricName("task-destruction-rate", metricGrpName, "The average per-second number of destructed tasks", metricTags), new Rate(new Count()));
|
||||
this.taskDestructionSensor.add(metrics.metricName("task-destruction-rate", metricGrpName, "The average per-second number of destructed tasks", metricTags), new Rate(new Count()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -733,11 +733,11 @@ public class StreamThread extends Thread {
|
|||
}
|
||||
|
||||
private void addLatencyMetrics(String metricGrpName, Sensor sensor, String entityName, String opName, Map<String, String> tags) {
|
||||
maybeAddMetric(sensor, new MetricName(opName + "-avg-latency-ms", metricGrpName,
|
||||
maybeAddMetric(sensor, metrics.metricName(opName + "-avg-latency-ms", metricGrpName,
|
||||
"The average latency in milliseconds of " + entityName + " " + opName + " operation.", tags), new Avg());
|
||||
maybeAddMetric(sensor, new MetricName(opName + "-max-latency-ms", metricGrpName,
|
||||
maybeAddMetric(sensor, metrics.metricName(opName + "-max-latency-ms", metricGrpName,
|
||||
"The max latency in milliseconds of " + entityName + " " + opName + " operation.", tags), new Max());
|
||||
maybeAddMetric(sensor, new MetricName(opName + "-qps", metricGrpName,
|
||||
maybeAddMetric(sensor, metrics.metricName(opName + "-qps", metricGrpName,
|
||||
"The average number of occurrence of " + entityName + " " + opName + " operation per second.", tags), new Rate(new Count()));
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Reference in New Issue