mirror of https://github.com/apache/druid.git
Merge pull request #773 from metamx/cleanup-metrics
Use multi-val dims for metrics
This commit is contained in:
commit
efa898422d
|
@ -19,14 +19,10 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class DataSourceUtil
|
||||
{
|
||||
public static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
|
||||
public static String getMetricName(DataSource dataSource)
|
||||
{
|
||||
final List<String> names = dataSource.getNames();
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.query;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QueryMetricUtil
|
||||
{
|
||||
public static <T> ServiceMetricEvent.Builder makeQueryTimeMetric(Query<T> query)
|
||||
{
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(query.getType())
|
||||
.setUser5(
|
||||
Lists.transform(
|
||||
query.getIntervals(),
|
||||
new Function<Interval, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(Interval input)
|
||||
{
|
||||
return input.toString();
|
||||
}
|
||||
}
|
||||
).toArray(new String[query.getIntervals().size()])
|
||||
)
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString());
|
||||
}
|
||||
|
||||
public static <T> ServiceMetricEvent.Builder makeRequestTimeMetric(
|
||||
final ObjectMapper jsonMapper, final Query<T> query, final String remoteAddr
|
||||
) throws JsonProcessingException
|
||||
{
|
||||
return makeQueryTimeMetric(query)
|
||||
.setUser3(
|
||||
jsonMapper.writeValueAsString(
|
||||
query.getContext() == null
|
||||
? ImmutableMap.of()
|
||||
: query.getContext()
|
||||
)
|
||||
)
|
||||
.setUser7(remoteAddr)
|
||||
.setUser8(query.getId());
|
||||
}
|
||||
}
|
|
@ -22,7 +22,6 @@ package io.druid.query.groupby;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -42,11 +41,11 @@ import io.druid.data.input.Row;
|
|||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.SubqueryQueryRunner;
|
||||
|
@ -58,8 +57,6 @@ import io.druid.query.filter.DimFilter;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
@ -210,19 +207,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(GroupByQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser3(String.format("%,d dims", query.getDimensions().size()))
|
||||
.setUser4("groupBy")
|
||||
.setUser5(Joiner.on(",").join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser3(String.format("%,d dims", query.getDimensions().size()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,12 +22,10 @@ package io.druid.query.metadata;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.MergeSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
|
@ -36,9 +34,8 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
|
@ -47,7 +44,6 @@ import io.druid.query.metadata.metadata.ColumnAnalysis;
|
|||
import io.druid.query.metadata.metadata.SegmentAnalysis;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -62,14 +58,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
};
|
||||
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
|
||||
|
||||
private final QueryConfig config;
|
||||
|
||||
@Inject
|
||||
public SegmentMetadataQueryQueryToolChest(QueryConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<SegmentAnalysis> mergeResults(final QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
|
@ -158,17 +146,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SegmentMetadataQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(query.getType())
|
||||
.setUser5(Joiner.on(",").join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -195,9 +173,9 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
{
|
||||
byte[] includerBytes = query.getToInclude().getCacheKey();
|
||||
return ByteBuffer.allocate(1 + includerBytes.length)
|
||||
.put(SEGMENT_METADATA_CACHE_PREFIX)
|
||||
.put(includerBytes)
|
||||
.array();
|
||||
.put(SEGMENT_METADATA_CACHE_PREFIX)
|
||||
.put(includerBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -39,9 +38,9 @@ import com.metamx.common.guava.nary.BinaryFn;
|
|||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -53,8 +52,6 @@ import io.druid.query.search.search.SearchHit;
|
|||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -67,7 +64,6 @@ import java.util.Set;
|
|||
public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResultValue>, SearchQuery>
|
||||
{
|
||||
private static final byte SEARCH_QUERY = 0x2;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Result<SearchResultValue>> TYPE_REFERENCE = new TypeReference<Result<SearchResultValue>>()
|
||||
{
|
||||
};
|
||||
|
@ -124,17 +120,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SearchQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("search")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,7 +167,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
|||
final ByteBuffer queryCacheKey = ByteBuffer
|
||||
.allocate(
|
||||
1 + 4 + granularityBytes.length + filterBytes.length +
|
||||
querySpecBytes.length + dimensionsBytesSize
|
||||
querySpecBytes.length + dimensionsBytesSize
|
||||
)
|
||||
.put(SEARCH_QUERY)
|
||||
.put(Ints.toByteArray(query.getLimit()))
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
|
@ -35,10 +34,10 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -47,8 +46,6 @@ import io.druid.query.ResultMergeQueryRunner;
|
|||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
|
@ -62,7 +59,6 @@ import java.util.Set;
|
|||
public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResultValue>, SelectQuery>
|
||||
{
|
||||
private static final byte SELECT_QUERY = 0x13;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
|
||||
new TypeReference<Object>()
|
||||
{
|
||||
|
@ -126,17 +122,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SelectQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("Select")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -261,13 +247,13 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
|
|||
new SelectResultValue(
|
||||
(Map<String, Integer>) jsonMapper.convertValue(
|
||||
resultIter.next(), new TypeReference<Map<String, Integer>>()
|
||||
{
|
||||
}
|
||||
{
|
||||
}
|
||||
),
|
||||
(List<EventHolder>) jsonMapper.convertValue(
|
||||
resultIter.next(), new TypeReference<List<EventHolder>>()
|
||||
{
|
||||
}
|
||||
{
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.timeseries;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -33,11 +32,11 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -48,8 +47,6 @@ import io.druid.query.aggregation.MetricManipulationFn;
|
|||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -62,7 +59,6 @@ import java.util.Map;
|
|||
public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
|
||||
{
|
||||
private static final byte TIMESERIES_QUERY = 0x0;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
|
||||
new TypeReference<Object>()
|
||||
{
|
||||
|
@ -124,18 +120,8 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(TimeseriesQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("timeseries")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.topn;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -37,10 +36,10 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -52,8 +51,6 @@ import io.druid.query.aggregation.MetricManipulationFn;
|
|||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
@ -65,7 +62,6 @@ import java.util.Map;
|
|||
public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultValue>, TopNQuery>
|
||||
{
|
||||
private static final byte TOPN_QUERY = 0x1;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Result<TopNResultValue>> TYPE_REFERENCE = new TypeReference<Result<TopNResultValue>>()
|
||||
{
|
||||
};
|
||||
|
@ -139,18 +135,15 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(TopNQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(String.format("topN/%s/%s", query.getThreshold(), query.getDimensionSpec().getDimension()))
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser4(
|
||||
String.format(
|
||||
"topN/%s/%s",
|
||||
query.getThreshold(),
|
||||
query.getDimensionSpec().getDimension()
|
||||
)
|
||||
)
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -98,7 +98,7 @@ public class SegmentAnalyzerTest
|
|||
{
|
||||
final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
|
||||
(QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(new QueryConfig()),
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
), index
|
||||
);
|
||||
|
|
|
@ -47,7 +47,7 @@ public class SegmentMetadataQueryTest
|
|||
@SuppressWarnings("unchecked")
|
||||
private final QueryRunner runner = makeQueryRunner(
|
||||
new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(new QueryConfig()),
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER)
|
||||
);
|
||||
private ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
|
|
@ -23,11 +23,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.guice.annotations.Smile;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.server.log.RequestLogger;
|
||||
import io.druid.server.router.QueryHostFinder;
|
||||
import io.druid.server.router.Router;
|
||||
|
@ -261,22 +260,8 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
|
|||
final long requestTime = System.currentTimeMillis() - start;
|
||||
try {
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser3(
|
||||
jsonMapper.writeValueAsString(
|
||||
query.getContext() == null
|
||||
? ImmutableMap.of()
|
||||
: query.getContext()
|
||||
)
|
||||
)
|
||||
.setUser4(query.getType())
|
||||
.setUser5(DataSourceUtil.COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(req.getRemoteAddr())
|
||||
.setUser8(query.getId())
|
||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
|
||||
.build("request/time", requestTime)
|
||||
QueryMetricUtil.makeRequestTimeMetric(jsonMapper, query, req.getRemoteAddr())
|
||||
.build("request/time", requestTime)
|
||||
);
|
||||
|
||||
requestLogger.log(
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.JsonGenerator;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.io.ByteStreams;
|
||||
|
@ -34,12 +33,11 @@ import com.metamx.common.guava.Yielder;
|
|||
import com.metamx.common.guava.YieldingAccumulator;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.guice.annotations.Smile;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryInterruptedException;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
import io.druid.server.initialization.ServerConfig;
|
||||
import io.druid.server.log.RequestLogger;
|
||||
|
@ -67,7 +65,6 @@ import java.util.UUID;
|
|||
public class QueryResource
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(QueryResource.class);
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
public static final String APPLICATION_SMILE = "application/smile";
|
||||
public static final String APPLICATION_JSON = "application/json";
|
||||
|
||||
|
@ -176,22 +173,8 @@ public class QueryResource
|
|||
try {
|
||||
long requestTime = System.currentTimeMillis() - start;
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser3(
|
||||
jsonMapper.writeValueAsString(
|
||||
query.getContext() == null
|
||||
? ImmutableMap.of()
|
||||
: query.getContext()
|
||||
)
|
||||
)
|
||||
.setUser4(query.getType())
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(req.getRemoteAddr())
|
||||
.setUser8(queryId)
|
||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
|
||||
.build("request/time", requestTime)
|
||||
QueryMetricUtil.makeRequestTimeMetric(jsonMapper, query, req.getRemoteAddr())
|
||||
.build("request/time", requestTime)
|
||||
);
|
||||
|
||||
requestLogger.log(
|
||||
|
|
Loading…
Reference in New Issue