mirror of
https://github.com/apache/druid.git
synced 2025-02-25 20:48:05 +00:00
finalize variables and optimize imports
This commit is contained in:
parent
be1ef3a161
commit
5601c51ae6
@ -19,7 +19,6 @@
|
||||
|
||||
package io.druid.query;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
|
@ -23,25 +23,14 @@ package io.druid.query;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
import io.druid.query.timeseries.TimeseriesQuery;
|
||||
import io.druid.query.topn.TopNQuery;
|
||||
|
||||
public class QueryDataSource implements DataSource
|
||||
{
|
||||
@JsonProperty
|
||||
private Query query;
|
||||
private final Query query;
|
||||
|
||||
public QueryDataSource()
|
||||
{
|
||||
}
|
||||
|
||||
public QueryDataSource(Query query)
|
||||
@JsonCreator
|
||||
public QueryDataSource(@JsonProperty("query") Query query)
|
||||
{
|
||||
this.query = query;
|
||||
}
|
||||
@ -51,11 +40,6 @@ public class QueryDataSource implements DataSource
|
||||
return query;
|
||||
}
|
||||
|
||||
public void setQuery(Query query)
|
||||
{
|
||||
this.query = query;
|
||||
}
|
||||
|
||||
public String toString() { return query.toString(); }
|
||||
|
||||
@Override
|
||||
|
@ -22,7 +22,6 @@
|
||||
package io.druid.query;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import org.joda.time.Period;
|
||||
|
||||
/**
|
||||
* If there's a subquery, run it instead of the outer query
|
||||
@ -42,8 +41,7 @@ public class SubqueryQueryRunner<T> implements QueryRunner<T>
|
||||
DataSource dataSource = query.getDataSource();
|
||||
if (dataSource instanceof QueryDataSource) {
|
||||
return run((Query<T>) ((QueryDataSource) dataSource).getQuery());
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
return baseRunner.run(query);
|
||||
}
|
||||
}
|
||||
|
@ -26,18 +26,12 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
public class TableDataSource implements DataSource
|
||||
{
|
||||
@JsonProperty
|
||||
private String name;
|
||||
private final String name;
|
||||
|
||||
@JsonCreator
|
||||
public TableDataSource()
|
||||
public TableDataSource(@JsonProperty("name") String name)
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public TableDataSource(String name)
|
||||
{
|
||||
this.name = name==null? name : name.toLowerCase();
|
||||
this.name = name.toLowerCase();
|
||||
}
|
||||
|
||||
public String getName()
|
||||
@ -45,11 +39,6 @@ public class TableDataSource implements DataSource
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name)
|
||||
{
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public String toString() { return name; }
|
||||
|
||||
@Override
|
||||
|
@ -32,7 +32,12 @@ import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
|
@ -24,9 +24,6 @@ import com.google.common.collect.Lists;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.Rows;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
|
@ -34,7 +34,13 @@ import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.SubqueryQueryRunner;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
@ -101,8 +107,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
||||
IncrementalIndexStorageAdapter adapter
|
||||
= new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult));
|
||||
result = engine.process(query, adapter);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
result = runner.run(query);
|
||||
}
|
||||
|
||||
|
@ -34,7 +34,11 @@ import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.metadata.metadata.ColumnAnalysis;
|
||||
import io.druid.query.metadata.metadata.SegmentAnalysis;
|
||||
|
@ -37,7 +37,14 @@ import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.ResultGranularTimestampComparator;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
|
@ -25,7 +25,10 @@ import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.search.SearchResultValue;
|
||||
import io.druid.query.spec.QuerySegmentSpec;
|
||||
|
@ -24,7 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import io.druid.query.spec.QuerySegmentSpec;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -31,7 +31,12 @@ import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BySegmentSkippingQueryRunner;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.timeline.LogicalSegment;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -24,7 +24,11 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
|
@ -22,8 +22,6 @@ package io.druid.query.timeseries;
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunnerHelper;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
|
@ -32,7 +32,16 @@ import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.ResultGranularTimestampComparator;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -22,7 +22,13 @@ package io.druid.query.timeseries;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.ChainedExecutionQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
|
||||
|
@ -24,7 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
|
@ -31,7 +31,11 @@ import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import io.druid.query.spec.QuerySegmentSpec;
|
||||
import io.druid.segment.*;
|
||||
import io.druid.segment.IncrementalIndexSegment;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.QueryableIndexSegment;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.TestIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -22,22 +22,14 @@
|
||||
package io.druid.query.search;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class SearchQueryTest
|
||||
{
|
||||
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -54,8 +54,6 @@ import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static io.druid.query.QueryRunnerTestHelper.*;
|
||||
|
||||
/**
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
|
@ -36,7 +36,13 @@ import org.junit.Test;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import static io.druid.query.QueryRunnerTestHelper.*;
|
||||
import static io.druid.query.QueryRunnerTestHelper.addRowsIndexConstant;
|
||||
import static io.druid.query.QueryRunnerTestHelper.allGran;
|
||||
import static io.druid.query.QueryRunnerTestHelper.commonAggregators;
|
||||
import static io.druid.query.QueryRunnerTestHelper.dataSource;
|
||||
import static io.druid.query.QueryRunnerTestHelper.fullOnInterval;
|
||||
import static io.druid.query.QueryRunnerTestHelper.indexMetric;
|
||||
import static io.druid.query.QueryRunnerTestHelper.providerDimension;
|
||||
|
||||
public class TopNQueryTest
|
||||
{
|
||||
|
@ -25,12 +25,16 @@ import com.google.common.collect.Ordering;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.client.selector.QueryableDruidServer;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.client.selector.ServerSelectorStrategy;
|
||||
import io.druid.concurrent.Execs;
|
||||
import io.druid.guice.annotations.Client;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChestWarehouse;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import io.druid.timeline.partition.PartitionChunk;
|
||||
@ -240,8 +244,7 @@ public class BrokerServerView implements TimelineServerView
|
||||
|
||||
if (dataSource instanceof TableDataSource) {
|
||||
table = ((TableDataSource) dataSource).getName();
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
throw new UnsupportedOperationException("Unsupported data source type: " + dataSource.getClass().getSimpleName());
|
||||
}
|
||||
|
||||
|
@ -40,8 +40,8 @@ import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.client.cache.Cache;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.client.selector.QueryableDruidServer;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.guice.annotations.Smile;
|
||||
import io.druid.query.BySegmentResultValueClass;
|
||||
import io.druid.query.CacheStrategy;
|
||||
@ -286,8 +286,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||
objectMapper.getFactory().createParser(cachedResult),
|
||||
cacheObjectClazz
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
} catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
@ -425,8 +424,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||
}
|
||||
|
||||
cache.put(key, valueBytes);
|
||||
}
|
||||
catch (IOException e) {
|
||||
} catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
@ -30,11 +30,19 @@ import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.query.*;
|
||||
import io.druid.segment.ReferenceCountingSegment;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.FinalizeResultsQueryRunner;
|
||||
import io.druid.query.NoopQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.segment.realtime.plumber.Plumber;
|
||||
import io.druid.segment.realtime.plumber.Sink;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Period;
|
||||
@ -90,6 +98,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
Closeables.closeQuietly(chief);
|
||||
}
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics(String datasource)
|
||||
{
|
||||
FireChief chief = chiefs.get(datasource);
|
||||
@ -125,8 +134,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
String dataSourceName;
|
||||
try {
|
||||
dataSourceName = ((TableDataSource) query.getDataSource()).getName();
|
||||
}
|
||||
catch (ClassCastException e) {
|
||||
} catch (ClassCastException e) {
|
||||
throw new UnsupportedOperationException("Subqueries are only supported in the broker");
|
||||
}
|
||||
return dataSourceName;
|
||||
@ -164,8 +172,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
log.info("Someone get us a plumber!");
|
||||
plumber = fireDepartment.findPlumber();
|
||||
log.info("We have our plumber!");
|
||||
}
|
||||
catch (IOException e) {
|
||||
} catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
@ -192,8 +199,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
try {
|
||||
try {
|
||||
inputRow = firehose.nextRow();
|
||||
}
|
||||
catch (Exception e) {
|
||||
} catch (Exception e) {
|
||||
log.debug(e, "thrown away line due to exception, considering unparseable");
|
||||
metrics.incrementUnparseable();
|
||||
continue;
|
||||
@ -218,8 +224,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
}
|
||||
metrics.incrementProcessed();
|
||||
}
|
||||
catch (FormattedException e) {
|
||||
} catch (FormattedException e) {
|
||||
log.info(e, "unparseable line: %s", e.getDetails());
|
||||
metrics.incrementUnparseable();
|
||||
continue;
|
||||
@ -235,8 +240,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||
.emit();
|
||||
normalExit = false;
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
} finally {
|
||||
Closeables.closeQuietly(firehose);
|
||||
if (normalExit) {
|
||||
plumber.finishJob();
|
||||
|
@ -30,7 +30,20 @@ import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.CountingMap;
|
||||
import io.druid.guice.annotations.Processing;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BySegmentQueryRunner;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.FinalizeResultsQueryRunner;
|
||||
import io.druid.query.MetricsEmittingQueryRunner;
|
||||
import io.druid.query.NoopQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.ReferenceCountingSegmentQueryRunner;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.spec.QuerySegmentSpec;
|
||||
import io.druid.query.spec.SpecificSegmentQueryRunner;
|
||||
import io.druid.query.spec.SpecificSegmentSpec;
|
||||
@ -228,8 +241,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||
String dataSourceName;
|
||||
try {
|
||||
dataSourceName = ((TableDataSource) query.getDataSource()).getName();
|
||||
}
|
||||
catch (ClassCastException e) {
|
||||
} catch (ClassCastException e) {
|
||||
throw new UnsupportedOperationException("Subqueries are only supported in the broker");
|
||||
}
|
||||
|
||||
@ -314,8 +326,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||
String dataSourceName;
|
||||
try {
|
||||
dataSourceName = ((TableDataSource) query.getDataSource()).getName();
|
||||
}
|
||||
catch (ClassCastException e) {
|
||||
} catch (ClassCastException e) {
|
||||
throw new UnsupportedOperationException("Subqueries are only supported in the broker");
|
||||
}
|
||||
|
||||
|
@ -23,7 +23,12 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
|
||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.*;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.guava.MergeIterable;
|
||||
@ -38,7 +43,17 @@ import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.granularity.PeriodGranularity;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.*;
|
||||
import io.druid.query.BySegmentResultValueClass;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.MapQueryToolChestWarehouse;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
@ -58,11 +73,19 @@ import io.druid.query.timeboundary.TimeBoundaryResultValue;
|
||||
import io.druid.query.timeseries.TimeseriesQuery;
|
||||
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||
import io.druid.query.topn.*;
|
||||
import io.druid.query.topn.TopNQuery;
|
||||
import io.druid.query.topn.TopNQueryBuilder;
|
||||
import io.druid.query.topn.TopNQueryConfig;
|
||||
import io.druid.query.topn.TopNQueryQueryToolChest;
|
||||
import io.druid.query.topn.TopNResultValue;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import io.druid.timeline.partition.*;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import io.druid.timeline.partition.PartitionChunk;
|
||||
import io.druid.timeline.partition.ShardSpec;
|
||||
import io.druid.timeline.partition.SingleElementPartitionChunk;
|
||||
import io.druid.timeline.partition.StringPartitionChunk;
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.DateTime;
|
||||
@ -77,7 +100,13 @@ import org.junit.runners.Parameterized;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
@ -552,7 +581,8 @@ public class CachingClusteredClientTest
|
||||
);
|
||||
}
|
||||
|
||||
public void testQueryCaching(final Query query, Object... args) {
|
||||
public void testQueryCaching(final Query query, Object... args)
|
||||
{
|
||||
testQueryCaching(3, true, query, args);
|
||||
}
|
||||
|
||||
@ -732,8 +762,7 @@ public class CachingClusteredClientTest
|
||||
Query capturedQuery = (Query) queryCapture.getValue();
|
||||
if (expectBySegment) {
|
||||
Assert.assertEquals("true", capturedQuery.getContextValue("bySegment"));
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
Assert.assertTrue(
|
||||
capturedQuery.getContextValue("bySegment") == null ||
|
||||
capturedQuery.getContextValue("bySegment").equals("false")
|
||||
@ -791,7 +820,8 @@ public class CachingClusteredClientTest
|
||||
}
|
||||
timeline.add(queryIntervals.get(k), String.valueOf(k), chunk);
|
||||
}
|
||||
} return serverExpectationList;
|
||||
}
|
||||
return serverExpectationList;
|
||||
}
|
||||
|
||||
private Sequence<Result<TimeseriesResultValue>> toQueryableTimeseriesResults(
|
||||
|
Loading…
x
Reference in New Issue
Block a user