Remove metrics emitting from caching clustered client

This commit is contained in:
Charles Allen 2015-06-25 15:42:46 -07:00
parent 0c85c8c60a
commit fbcac10e00
3 changed files with 3 additions and 26 deletions

View File

@ -29,7 +29,6 @@ Available Metrics
|Metric|Description|Dimensions|Normal Value|
|------|-----------|----------|------------|
|`query/time`|Milliseconds taken to complete a query.|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|< 1s|
|`query/node/time`|Milliseconds taken to query individual historical/realtime nodes.|id, status, server.|< 1s|
|`query/intervalChunk/time`|Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk.|id, status, chunkInterval (if interval chunking is enabled).|< 1s|
### Historical

View File

@ -44,7 +44,6 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.client.cache.Cache;
import io.druid.client.cache.CacheConfig;
import io.druid.client.selector.QueryableDruidServer;
@ -53,7 +52,6 @@ import io.druid.guice.annotations.BackgroundCaching;
import io.druid.guice.annotations.Smile;
import io.druid.query.BySegmentResultValueClass;
import io.druid.query.CacheStrategy;
import io.druid.query.MetricsEmittingQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryToolChest;
@ -69,7 +67,6 @@ import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.partition.PartitionChunk;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@ -94,7 +91,6 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
private final ObjectMapper objectMapper;
private final CacheConfig cacheConfig;
private final ListeningExecutorService backgroundExecutorService;
private final ServiceEmitter emitter;
@Inject
public CachingClusteredClient(
@ -103,8 +99,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
Cache cache,
@Smile ObjectMapper objectMapper,
@BackgroundCaching ExecutorService backgroundExecutorService,
CacheConfig cacheConfig,
ServiceEmitter emitter
CacheConfig cacheConfig
)
{
this.warehouse = warehouse;
@ -113,7 +108,6 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
this.objectMapper = objectMapper;
this.cacheConfig = cacheConfig;
this.backgroundExecutorService = MoreExecutors.listeningDecorator(backgroundExecutorService);
this.emitter = emitter;
serverView.registerSegmentCallback(
Executors.newFixedThreadPool(
@ -332,20 +326,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
final DruidServer server = entry.getKey();
final List<SegmentDescriptor> descriptors = entry.getValue();
final QueryRunner clientQueryable = new MetricsEmittingQueryRunner(
emitter,
new Function<Query<T>, ServiceMetricEvent.Builder>()
{
@Override
public ServiceMetricEvent.Builder apply(@Nullable final Query<T> input)
{
return toolChest.makeMetricBuilder(input);
}
},
serverView.getQueryRunner(server),
"query/node/time",
ImmutableMap.of("server",server.getName())
);
final QueryRunner clientQueryable = serverView.getQueryRunner(server);
if (clientQueryable == null) {
log.error("WTF!? server[%s] doesn't have a client Queryable?", server);

View File

@ -217,7 +217,6 @@ public class CachingClusteredClientTest
protected VersionedIntervalTimeline<String, ServerSelector> timeline;
protected TimelineServerView serverView;
protected Cache cache;
protected ServiceEmitter emitter;
DruidServer[] servers;
public CachingClusteredClientTest(int randomSeed)
@ -247,7 +246,6 @@ public class CachingClusteredClientTest
timeline = new VersionedIntervalTimeline<>(Ordering.<String>natural());
serverView = EasyMock.createStrictMock(TimelineServerView.class);
cache = MapCache.create(100000);
emitter = EasyMock.createStrictMock(ServiceEmitter.class);
client = makeClient(MoreExecutors.sameThreadExecutor());
servers = new DruidServer[]{
@ -2097,8 +2095,7 @@ public class CachingClusteredClientTest
{
return true;
}
},
emitter
}
);
}