mirror of https://github.com/apache/druid.git
fix issue where result level cache was recomputing post aggs that were already cached, causing issues with finalizing aggregators (#7708)
This commit is contained in:
parent
ddda8b74cb
commit
c4a4223c9c
|
@ -294,8 +294,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
||||||
private final List<AggregatorFactory> aggs = Lists.newArrayList(query.getAggregatorSpecs());
|
private final List<AggregatorFactory> aggs = Lists.newArrayList(query.getAggregatorSpecs());
|
||||||
private final List<PostAggregator> postAggs = AggregatorUtil.pruneDependentPostAgg(
|
private final List<PostAggregator> postAggs = AggregatorUtil.pruneDependentPostAgg(
|
||||||
query.getPostAggregatorSpecs(),
|
query.getPostAggregatorSpecs(),
|
||||||
query.getTopNMetricSpec()
|
query.getTopNMetricSpec().getMetricName(query.getDimensionSpec())
|
||||||
.getMetricName(query.getDimensionSpec())
|
|
||||||
);
|
);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -419,14 +418,15 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
||||||
for (PostAggregator postAgg : postAggs) {
|
|
||||||
vals.put(postAgg.getName(), postAgg.compute(vals));
|
|
||||||
}
|
|
||||||
if (isResultLevelCache) {
|
if (isResultLevelCache) {
|
||||||
Iterator<PostAggregator> postItr = query.getPostAggregatorSpecs().iterator();
|
Iterator<PostAggregator> postItr = query.getPostAggregatorSpecs().iterator();
|
||||||
while (postItr.hasNext() && resultIter.hasNext()) {
|
while (postItr.hasNext() && resultIter.hasNext()) {
|
||||||
vals.put(postItr.next().getName(), resultIter.next());
|
vals.put(postItr.next().getName(), resultIter.next());
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
for (PostAggregator postAgg : postAggs) {
|
||||||
|
vals.put(postAgg.getName(), postAgg.compute(vals));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
retVal.add(vals);
|
retVal.add(vals);
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.collections.CloseableStupidPool;
|
import org.apache.druid.collections.CloseableStupidPool;
|
||||||
import org.apache.druid.collections.SerializablePair;
|
import org.apache.druid.collections.SerializablePair;
|
||||||
|
import org.apache.druid.hll.HyperLogLogCollector;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
@ -40,6 +41,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.SerializablePairLongString;
|
import org.apache.druid.query.aggregation.SerializablePairLongString;
|
||||||
|
import org.apache.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||||
|
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.last.DoubleLastAggregatorFactory;
|
import org.apache.druid.query.aggregation.last.DoubleLastAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.last.FloatLastAggregatorFactory;
|
import org.apache.druid.query.aggregation.last.FloatLastAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.last.LongLastAggregatorFactory;
|
import org.apache.druid.query.aggregation.last.LongLastAggregatorFactory;
|
||||||
|
@ -47,6 +50,7 @@ import org.apache.druid.query.aggregation.last.StringLastAggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||||
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
|
import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||||
|
import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator;
|
||||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
import org.apache.druid.segment.IncrementalIndexSegment;
|
import org.apache.druid.segment.IncrementalIndexSegment;
|
||||||
|
@ -79,6 +83,15 @@ public class TopNQueryQueryToolChestTest
|
||||||
doTestCacheStrategy(ValueType.LONG, 2L);
|
doTestCacheStrategy(ValueType.LONG, 2L);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCacheStrategyOrderByPostAggs() throws Exception
|
||||||
|
{
|
||||||
|
doTestCacheStrategyOrderByPost(ValueType.STRING, "val1");
|
||||||
|
doTestCacheStrategyOrderByPost(ValueType.FLOAT, 2.1f);
|
||||||
|
doTestCacheStrategyOrderByPost(ValueType.DOUBLE, 2.1d);
|
||||||
|
doTestCacheStrategyOrderByPost(ValueType.LONG, 2L);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testComputeCacheKeyWithDifferentPostAgg()
|
public void testComputeCacheKeyWithDifferentPostAgg()
|
||||||
{
|
{
|
||||||
|
@ -306,6 +319,28 @@ public class TopNQueryQueryToolChestTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HyperLogLogCollector getIntermediateHllCollector(final ValueType valueType, final Object dimValue)
|
||||||
|
{
|
||||||
|
HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
|
||||||
|
switch (valueType) {
|
||||||
|
case LONG:
|
||||||
|
collector.add(CardinalityAggregator.hashFn.hashLong((Long) dimValue).asBytes());
|
||||||
|
break;
|
||||||
|
case DOUBLE:
|
||||||
|
collector.add(CardinalityAggregator.hashFn.hashLong(Double.doubleToLongBits((Double) dimValue)).asBytes());
|
||||||
|
break;
|
||||||
|
case FLOAT:
|
||||||
|
collector.add(CardinalityAggregator.hashFn.hashInt(Float.floatToIntBits((Float) dimValue)).asBytes());
|
||||||
|
break;
|
||||||
|
case STRING:
|
||||||
|
collector.add(CardinalityAggregator.hashFn.hashUnencodedChars((String) dimValue).asBytes());
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("bad valueType: " + valueType);
|
||||||
|
}
|
||||||
|
return collector;
|
||||||
|
}
|
||||||
|
|
||||||
private void doTestCacheStrategy(final ValueType valueType, final Object dimValue) throws IOException
|
private void doTestCacheStrategy(final ValueType valueType, final Object dimValue) throws IOException
|
||||||
{
|
{
|
||||||
CacheStrategy<Result<TopNResultValue>, Object, TopNQuery> strategy =
|
CacheStrategy<Result<TopNResultValue>, Object, TopNQuery> strategy =
|
||||||
|
@ -419,6 +454,102 @@ public class TopNQueryQueryToolChestTest
|
||||||
Assert.assertEquals(typeAdjustedResult2, fromResultCacheResult);
|
Assert.assertEquals(typeAdjustedResult2, fromResultCacheResult);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void doTestCacheStrategyOrderByPost(final ValueType valueType, final Object dimValue) throws IOException
|
||||||
|
{
|
||||||
|
CacheStrategy<Result<TopNResultValue>, Object, TopNQuery> strategy =
|
||||||
|
new TopNQueryQueryToolChest(null, null).getCacheStrategy(
|
||||||
|
new TopNQuery(
|
||||||
|
new TableDataSource("dummy"),
|
||||||
|
VirtualColumns.EMPTY,
|
||||||
|
new DefaultDimensionSpec("test", "test", valueType),
|
||||||
|
new NumericTopNMetricSpec("post"),
|
||||||
|
3,
|
||||||
|
new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2015-01-01/2015-01-02"))),
|
||||||
|
null,
|
||||||
|
Granularities.ALL,
|
||||||
|
ImmutableList.of(
|
||||||
|
new HyperUniquesAggregatorFactory("metric1", "test", false, false),
|
||||||
|
new CountAggregatorFactory("metric2")
|
||||||
|
),
|
||||||
|
ImmutableList.of(
|
||||||
|
new ArithmeticPostAggregator(
|
||||||
|
"post",
|
||||||
|
"+",
|
||||||
|
ImmutableList.of(
|
||||||
|
new FinalizingFieldAccessPostAggregator(
|
||||||
|
"metric1",
|
||||||
|
"metric1"
|
||||||
|
),
|
||||||
|
new FieldAccessPostAggregator(
|
||||||
|
"metric2",
|
||||||
|
"metric2"
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
),
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
HyperLogLogCollector collector = getIntermediateHllCollector(valueType, dimValue);
|
||||||
|
|
||||||
|
final Result<TopNResultValue> result1 = new Result<>(
|
||||||
|
// test timestamps that result in integer size millis
|
||||||
|
DateTimes.utc(123L),
|
||||||
|
new TopNResultValue(
|
||||||
|
Collections.singletonList(
|
||||||
|
ImmutableMap.of(
|
||||||
|
"test", dimValue,
|
||||||
|
"metric1", collector,
|
||||||
|
"metric2", 2,
|
||||||
|
"post", collector.estimateCardinality() + 2
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Object preparedValue = strategy.prepareForSegmentLevelCache().apply(
|
||||||
|
result1
|
||||||
|
);
|
||||||
|
|
||||||
|
ObjectMapper objectMapper = TestHelper.makeJsonMapper();
|
||||||
|
Object fromCacheValue = objectMapper.readValue(
|
||||||
|
objectMapper.writeValueAsBytes(preparedValue),
|
||||||
|
strategy.getCacheObjectClazz()
|
||||||
|
);
|
||||||
|
|
||||||
|
Result<TopNResultValue> fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue);
|
||||||
|
|
||||||
|
Assert.assertEquals(result1, fromCacheResult);
|
||||||
|
|
||||||
|
final Result<TopNResultValue> resultLevelCacheResult = new Result<>(
|
||||||
|
// test timestamps that result in integer size millis
|
||||||
|
DateTimes.utc(123L),
|
||||||
|
new TopNResultValue(
|
||||||
|
Collections.singletonList(
|
||||||
|
ImmutableMap.of(
|
||||||
|
"test", dimValue,
|
||||||
|
"metric1", collector.estimateCardinality(),
|
||||||
|
"metric2", 2,
|
||||||
|
"post", collector.estimateCardinality() + 2
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Object preparedResultCacheValue = strategy.prepareForCache(true).apply(
|
||||||
|
resultLevelCacheResult
|
||||||
|
);
|
||||||
|
|
||||||
|
Object fromResultCacheValue = objectMapper.readValue(
|
||||||
|
objectMapper.writeValueAsBytes(preparedResultCacheValue),
|
||||||
|
strategy.getCacheObjectClazz()
|
||||||
|
);
|
||||||
|
|
||||||
|
Result<TopNResultValue> fromResultCacheResult = strategy.pullFromCache(true).apply(fromResultCacheValue);
|
||||||
|
Assert.assertEquals(resultLevelCacheResult, fromResultCacheResult);
|
||||||
|
}
|
||||||
|
|
||||||
static class MockQueryRunner implements QueryRunner<Result<TopNResultValue>>
|
static class MockQueryRunner implements QueryRunner<Result<TopNResultValue>>
|
||||||
{
|
{
|
||||||
private final QueryRunner<Result<TopNResultValue>> runner;
|
private final QueryRunner<Result<TopNResultValue>> runner;
|
||||||
|
|
Loading…
Reference in New Issue