mirror of https://github.com/apache/druid.git
fix bug with previousStop not being respected for inverse lexi topN
This commit is contained in:
parent
b9015cee34
commit
9f62589d71
|
@ -31,7 +31,6 @@ import io.druid.segment.DimensionSelector;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -41,7 +40,6 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
|
|||
{
|
||||
private final Capabilities capabilities;
|
||||
private final TopNQuery query;
|
||||
private final Comparator<?> comparator;
|
||||
private final StupidPool<ByteBuffer> bufferPool;
|
||||
|
||||
public AggregateTopNMetricFirstAlgorithm(
|
||||
|
@ -52,8 +50,6 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
|
|||
{
|
||||
this.capabilities = capabilities;
|
||||
this.query = query;
|
||||
this.comparator = query.getTopNMetricSpec()
|
||||
.getComparator(query.getAggregatorSpecs(), query.getPostAggregatorSpecs());
|
||||
this.bufferPool = bufferPool;
|
||||
}
|
||||
|
||||
|
|
|
@ -242,8 +242,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
|||
query.getThreshold(),
|
||||
comparator,
|
||||
query.getAggregatorSpecs(),
|
||||
query.getPostAggregatorSpecs(),
|
||||
true
|
||||
query.getPostAggregatorSpecs()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,8 +77,7 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec
|
|||
int threshold,
|
||||
Comparator comparator,
|
||||
List<AggregatorFactory> aggFactories,
|
||||
List<PostAggregator> postAggs,
|
||||
boolean optimizeResultStorage
|
||||
List<PostAggregator> postAggs
|
||||
)
|
||||
{
|
||||
return delegate.getResultBuilder(
|
||||
|
@ -87,8 +86,7 @@ public class InvertedTopNMetricSpec implements TopNMetricSpec
|
|||
threshold,
|
||||
comparator,
|
||||
aggFactories,
|
||||
postAggs,
|
||||
canBeOptimizedUnordered()
|
||||
postAggs
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -82,8 +82,7 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec
|
|||
int threshold,
|
||||
Comparator comparator,
|
||||
List<AggregatorFactory> aggFactories,
|
||||
List<PostAggregator> postAggs,
|
||||
boolean optimizeResultStorage
|
||||
List<PostAggregator> postAggs
|
||||
)
|
||||
{
|
||||
return new TopNLexicographicResultBuilder(
|
||||
|
@ -92,8 +91,7 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec
|
|||
threshold,
|
||||
previousStop,
|
||||
comparator,
|
||||
aggFactories,
|
||||
optimizeResultStorage
|
||||
aggFactories
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -123,8 +123,7 @@ public class NumericTopNMetricSpec implements TopNMetricSpec
|
|||
int threshold,
|
||||
Comparator comparator,
|
||||
List<AggregatorFactory> aggFactories,
|
||||
List<PostAggregator> postAggs,
|
||||
boolean optimizeResultStorage
|
||||
List<PostAggregator> postAggs
|
||||
)
|
||||
{
|
||||
return new TopNNumericResultBuilder(timestamp, dimSpec, metric, threshold, comparator, aggFactories, postAggs);
|
||||
|
|
|
@ -129,8 +129,7 @@ public class TopNBinaryFn implements BinaryFn<Result<TopNResultValue>, Result<To
|
|||
threshold,
|
||||
comparator,
|
||||
aggregations,
|
||||
postAggregations,
|
||||
true
|
||||
postAggregations
|
||||
);
|
||||
for (DimensionAndMetricValueExtractor extractor : retVals.values()) {
|
||||
bob.addEntry(extractor);
|
||||
|
|
|
@ -39,8 +39,8 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder
|
|||
private final DateTime timestamp;
|
||||
private final DimensionSpec dimSpec;
|
||||
private final String previousStop;
|
||||
private final Comparator comparator;
|
||||
private final List<AggregatorFactory> aggFactories;
|
||||
private final boolean optimizeResultStorage;
|
||||
private MinMaxPriorityQueue<DimValHolder> pQueue = null;
|
||||
|
||||
public TopNLexicographicResultBuilder(
|
||||
|
@ -49,15 +49,14 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder
|
|||
int threshold,
|
||||
String previousStop,
|
||||
final Comparator comparator,
|
||||
List<AggregatorFactory> aggFactories,
|
||||
boolean optimizeResultStorage
|
||||
List<AggregatorFactory> aggFactories
|
||||
)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.dimSpec = dimSpec;
|
||||
this.previousStop = previousStop;
|
||||
this.comparator = comparator;
|
||||
this.aggFactories = aggFactories;
|
||||
this.optimizeResultStorage = optimizeResultStorage;
|
||||
|
||||
instantiatePQueue(threshold, comparator);
|
||||
}
|
||||
|
@ -71,7 +70,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder
|
|||
{
|
||||
Map<String, Object> metricValues = Maps.newLinkedHashMap();
|
||||
|
||||
if (!optimizeResultStorage || dimName.compareTo(previousStop) > 0) {
|
||||
if (comparator.compare(dimName, previousStop) > 0) {
|
||||
metricValues.put(dimSpec.getOutputName(), dimName);
|
||||
Iterator<AggregatorFactory> aggsIter = aggFactories.iterator();
|
||||
for (Object metricVal : metricVals) {
|
||||
|
|
|
@ -50,8 +50,7 @@ public interface TopNMetricSpec
|
|||
int threshold,
|
||||
Comparator comparator,
|
||||
List<AggregatorFactory> aggFactories,
|
||||
List<PostAggregator> postAggs,
|
||||
boolean optimizeResultStorage
|
||||
List<PostAggregator> postAggs
|
||||
);
|
||||
|
||||
public byte[] getCacheKey();
|
||||
|
|
|
@ -1097,12 +1097,46 @@ public class TopNQueryRunnerTest
|
|||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
providerDimension, "upfront",
|
||||
providerDimension, "total_market",
|
||||
"rows", 4L,
|
||||
"index", 4875.669677734375D,
|
||||
"addRowsIndexConstant", 4880.669677734375D,
|
||||
"index", 5351.814697265625D,
|
||||
"addRowsIndexConstant", 5356.814697265625D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_2
|
||||
),
|
||||
ImmutableMap.<String, Object>of(
|
||||
providerDimension, "spot",
|
||||
"rows", 18L,
|
||||
"index", 2231.8768157958984D,
|
||||
"addRowsIndexConstant", 2250.8768157958984D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNInvertedLexicographicWithNonExistingPreviousStop()
|
||||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.dimension(providerDimension)
|
||||
.metric(new InvertedTopNMetricSpec(new LexicographicTopNMetricSpec("u")))
|
||||
.threshold(4)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
providerDimension, "total_market",
|
||||
"rows", 4L,
|
||||
|
|
Loading…
Reference in New Issue