Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

SOLR-8785: Use Metrics library for core metrics #25

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 3 additions & 3 deletions lucene/ivy-versions.properties
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,9 @@ com.carrotsearch.randomizedtesting.version = 2.3.2

/com.carrotsearch/hppc = 0.7.1

com.codahale.metrics.version = 3.0.1
/com.codahale.metrics/metrics-core = ${com.codahale.metrics.version}
/com.codahale.metrics/metrics-healthchecks = ${com.codahale.metrics.version}
io.dropwizard.metrics.version = 3.1.2
/io.dropwizard.metrics/metrics-core = ${io.dropwizard.metrics.version}
/io.dropwizard.metrics/metrics-healthchecks = ${io.dropwizard.metrics.version}

/com.cybozu.labs/langdetect = 1.1-20120112
/com.drewnoakes/metadata-extractor = 2.6.2
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,14 @@
*/
package org.apache.solr.analytics.plugin;

import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;

import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.util.stats.Snapshot;
import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.TimerContext;
import org.apache.solr.util.stats.Metrics;

public class AnalyticsStatisticsCollector {
private final AtomicLong numRequests;
Expand All @@ -35,7 +36,7 @@ public class AnalyticsStatisticsCollector {
private final AtomicLong numQueries;
private final Timer requestTimes;

public TimerContext currentTimer;
public Timer.Context currentTimer;

public AnalyticsStatisticsCollector() {
numRequests = new AtomicLong();
Expand Down Expand Up @@ -97,17 +98,7 @@ public NamedList<Object> getStatistics() {
lst.add("rangeFacets", numRangeFacets.longValue());
lst.add("queryFacets", numQueryFacets.longValue());
lst.add("queriesInQueryFacets", numQueries.longValue());
lst.add("totalTime", requestTimes.getSum());
lst.add("avgRequestsPerSecond", requestTimes.getMeanRate());
lst.add("5minRateReqsPerSecond", requestTimes.getFiveMinuteRate());
lst.add("15minRateReqsPerSecond", requestTimes.getFifteenMinuteRate());
lst.add("avgTimePerRequest", requestTimes.getMean());
lst.add("medianRequestTime", snapshot.getMedian());
lst.add("75thPcRequestTime", snapshot.get75thPercentile());
lst.add("95thPcRequestTime", snapshot.get95thPercentile());
lst.add("99thPcRequestTime", snapshot.get99thPercentile());
lst.add("999thPcRequestTime", snapshot.get999thPercentile());
Metrics.addTimerMetrics(lst, requestTimes);
return lst;
}

}
4 changes: 2 additions & 2 deletions solr/contrib/morphlines-core/ivy.xml
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@

<dependency org="org.kitesdk" name="kite-morphlines-avro" rev="${/org.kitesdk/kite-morphlines-avro}" conf="compile" />

<dependency org="com.codahale.metrics" name="metrics-core" rev="${/com.codahale.metrics/metrics-core}" conf="compile" />
<dependency org="com.codahale.metrics" name="metrics-healthchecks" rev="${/com.codahale.metrics/metrics-healthchecks}" conf="compile" />
<dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="compile" />
<dependency org="io.dropwizard.metrics" name="metrics-healthchecks" rev="${/io.dropwizard.metrics/metrics-healthchecks}" conf="compile" />
<dependency org="com.typesafe" name="config" rev="${/com.typesafe/config}" conf="compile" />

<!-- Test Dependencies -->
Expand Down
1 change: 1 addition & 0 deletions solr/core/ivy.xml
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
<dependency org="log4j" name="log4j" rev="${/log4j/log4j}" conf="compile"/>
<dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="compile"/>
<dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
<dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="compile" />

<dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test"/>
<dependency org="cglib" name="cglib-nodep" rev="${/cglib/cglib-nodep}" conf="test"/>
Expand Down
14 changes: 7 additions & 7 deletions solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,13 @@
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;

import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
Expand Down Expand Up @@ -123,7 +123,7 @@ public DistributedQueue(SolrZkClient zookeeper, String dir, Overseer.Stats stats
* @return data at the first element of the queue, or null.
*/
public byte[] peek() throws KeeperException, InterruptedException {
TimerContext time = stats.time(dir + "_peek");
Timer.Context time = stats.time(dir + "_peek");
try {
return firstElement();
} finally {
Expand Down Expand Up @@ -151,7 +151,7 @@ public byte[] peek(boolean block) throws KeeperException, InterruptedException {
*/
public byte[] peek(long wait) throws KeeperException, InterruptedException {
Preconditions.checkArgument(wait > 0);
TimerContext time;
Timer.Context time;
if (wait == Long.MAX_VALUE) {
time = stats.time(dir + "_peek_wait_forever");
} else {
Expand Down Expand Up @@ -181,7 +181,7 @@ public byte[] peek(long wait) throws KeeperException, InterruptedException {
* @return Head of the queue or null.
*/
public byte[] poll() throws KeeperException, InterruptedException {
TimerContext time = stats.time(dir + "_poll");
Timer.Context time = stats.time(dir + "_poll");
try {
return removeFirst();
} finally {
Expand All @@ -195,7 +195,7 @@ public byte[] poll() throws KeeperException, InterruptedException {
* @return The former head of the queue
*/
public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
TimerContext time = stats.time(dir + "_remove");
Timer.Context time = stats.time(dir + "_remove");
try {
byte[] result = removeFirst();
if (result == null) {
Expand All @@ -214,7 +214,7 @@ public byte[] remove() throws NoSuchElementException, KeeperException, Interrupt
*/
public byte[] take() throws KeeperException, InterruptedException {
// Same as for element. Should refactor this.
TimerContext timer = stats.time(dir + "_take");
Timer.Context timer = stats.time(dir + "_take");
updateLock.lockInterruptibly();
try {
while (true) {
Expand All @@ -238,7 +238,7 @@ public byte[] take() throws KeeperException, InterruptedException {
* element to become visible.
*/
public void offer(byte[] data) throws KeeperException, InterruptedException {
TimerContext time = stats.time(dir + "_offer");
Timer.Context time = stats.time(dir + "_offer");
try {
while (true) {
try {
Expand Down
12 changes: 5 additions & 7 deletions solr/core/src/java/org/apache/solr/cloud/Overseer.java
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;

import com.codahale.metrics.Timer;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
Expand All @@ -59,9 +60,6 @@
import org.apache.solr.handler.admin.CollectionsHandler;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.stats.Clock;
import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
Expand Down Expand Up @@ -273,7 +271,7 @@ public void run() {
private ClusterState processQueueItem(ZkNodeProps message, ClusterState clusterState, ZkStateWriter zkStateWriter, boolean enableBatching, ZkStateWriter.ZkWriteCallback callback) throws Exception {
final String operation = message.getStr(QUEUE_OPERATION);
List<ZkWriteCommand> zkWriteCommands = null;
final TimerContext timerContext = stats.time(operation);
final Timer.Context timerContext = stats.time(operation);
try {
zkWriteCommands = processMessage(clusterState, message, operation);
stats.success(operation);
Expand Down Expand Up @@ -411,7 +409,7 @@ private List<ZkWriteCommand> processMessage(ClusterState clusterState,
}

private LeaderStatus amILeader() {
TimerContext timerContext = stats.time("am_i_leader");
Timer.Context timerContext = stats.time("am_i_leader");
boolean success = true;
try {
ZkNodeProps props = ZkNodeProps.load(zkClient.getData(
Expand Down Expand Up @@ -1115,7 +1113,7 @@ public void error(String operation) {
stat.errors.incrementAndGet();
}

public TimerContext time(String operation) {
public Timer.Context time(String operation) {
String op = operation.toLowerCase(Locale.ROOT);
Stat stat = stats.get(op);
if (stat == null) {
Expand Down Expand Up @@ -1173,7 +1171,7 @@ public static class Stat {
public Stat() {
this.success = new AtomicInteger();
this.errors = new AtomicInteger();
this.requestTime = new Timer(TimeUnit.MILLISECONDS, TimeUnit.MINUTES, Clock.defaultClock());
this.requestTime = new Timer();
this.failureDetails = new LinkedList<>();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;

import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
Expand Down Expand Up @@ -82,8 +84,7 @@
import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TimeOut;
import org.apache.solr.util.stats.Snapshot;
import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.Metrics;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
Expand Down Expand Up @@ -429,17 +430,7 @@ private void getOverseerStatus(ZkNodeProps message, NamedList results) throws Ke
lst.add("errors", errors);
}
Timer timer = entry.getValue().requestTime;
Snapshot snapshot = timer.getSnapshot();
lst.add("totalTime", timer.getSum());
lst.add("avgRequestsPerMinute", timer.getMeanRate());
lst.add("5minRateRequestsPerMinute", timer.getFiveMinuteRate());
lst.add("15minRateRequestsPerMinute", timer.getFifteenMinuteRate());
lst.add("avgTimePerRequest", timer.getMean());
lst.add("medianRequestTime", snapshot.getMedian());
lst.add("75thPctlRequestTime", snapshot.get75thPercentile());
lst.add("95thPctlRequestTime", snapshot.get95thPercentile());
lst.add("99thPctlRequestTime", snapshot.get99thPercentile());
lst.add("999thPctlRequestTime", snapshot.get999thPercentile());
Metrics.addTimerMetrics(lst, timer);
}
results.add("overseer_operations", overseerStats);
results.add("collection_operations", collectionStats);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;

import com.codahale.metrics.Timer;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
import org.apache.solr.cloud.Overseer.LeaderStatus;
Expand All @@ -39,7 +40,6 @@
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandlerFactory;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
Expand Down Expand Up @@ -352,7 +352,7 @@ public static String getLeaderId(SolrZkClient zkClient) throws KeeperException,I

protected LeaderStatus amILeader() {
String statsName = "collection_am_i_leader";
TimerContext timerContext = stats.time(statsName);
Timer.Context timerContext = stats.time(statsName);
boolean success = true;
try {
ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(
Expand Down Expand Up @@ -425,7 +425,7 @@ public Runner(OverseerMessageHandler messageHandler, ZkNodeProps message, String

public void run() {
String statsName = messageHandler.getTimerName(operation);
final TimerContext timerContext = stats.time(statsName);
final Timer.Context timerContext = stats.time(statsName);

boolean success = false;
final String asyncId = message.getStr(ASYNC);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,9 @@
import java.util.Set;
import java.util.TreeSet;

import com.codahale.metrics.Timer;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
Expand Down Expand Up @@ -86,7 +86,7 @@ public boolean containsTaskWithRequestId(String requestIdKey, String requestId)
*/
public byte[] remove(QueueEvent event) throws KeeperException,
InterruptedException {
TimerContext time = stats.time(dir + "_remove_event");
Timer.Context time = stats.time(dir + "_remove_event");
try {
String path = event.getId();
String responsePath = dir + "/" + response_prefix
Expand Down Expand Up @@ -181,7 +181,7 @@ private String createData(String path, byte[] data, CreateMode mode)
*/
public QueueEvent offer(byte[] data, long timeout) throws KeeperException,
InterruptedException {
TimerContext time = stats.time(dir + "_offer");
Timer.Context time = stats.time(dir + "_offer");
try {
// Create and watch the response node before creating the request node;
// otherwise we may miss the response.
Expand Down Expand Up @@ -218,7 +218,7 @@ public List<QueueEvent> peekTopN(int n, Set<String> excludeSet, long waitMillis)
ArrayList<QueueEvent> topN = new ArrayList<>();

LOG.debug("Peeking for top {} elements. ExcludeSet: {}", n, excludeSet);
TimerContext time = null;
Timer.Context time = null;
if (waitMillis == Long.MAX_VALUE) time = stats.time(dir + "_peekTopN_wait_forever");
else time = stats.time(dir + "_peekTopN_wait" + waitMillis);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,12 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;

import com.codahale.metrics.Timer;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
Expand Down Expand Up @@ -211,7 +211,7 @@ public ClusterState writePendingUpdates() throws IllegalStateException, KeeperEx
throw new IllegalStateException("ZkStateWriter has seen a tragic error, this instance can no longer be used");
}
if (!hasPendingUpdates()) return clusterState;
TimerContext timerContext = stats.time("update_state");
Timer.Context timerContext = stats.time("update_state");
boolean success = false;
try {
if (!updates.isEmpty()) {
Expand Down
Loading