mirror of https://github.com/apache/druid.git
fix SQL issue for group by queries with time filter that gets optimized to false (#10968)
* fix SQL issue for group by queries with time filter that gets optimized to false * short circuit always false in CombineAndSimplifyBounds * adjust * javadocs * add preconditions for and/or filters to ensure they have children * add comments, remove preconditions
This commit is contained in:
parent
9c083783c9
commit
58294329b7
|
@ -345,6 +345,8 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
/**
|
||||
* If this query has a single universal timestamp, return it. Otherwise return null.
|
||||
*
|
||||
* If {@link #getIntervals()} is empty, there are no results (or timestamps) so this method returns null.
|
||||
*
|
||||
* This method will return a nonnull timestamp in the following two cases:
|
||||
*
|
||||
* 1) CTX_KEY_FUDGE_TIMESTAMP is set (in which case this timestamp will be returned).
|
||||
|
@ -715,7 +717,12 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
if (!timestampStringFromContext.isEmpty()) {
|
||||
return DateTimes.utc(Long.parseLong(timestampStringFromContext));
|
||||
} else if (Granularities.ALL.equals(granularity)) {
|
||||
final DateTime timeStart = getIntervals().get(0).getStart();
|
||||
final List<Interval> intervals = getIntervals();
|
||||
if (intervals.isEmpty()) {
|
||||
// null, the "universal timestamp" of nothing
|
||||
return null;
|
||||
}
|
||||
final DateTime timeStart = intervals.get(0).getStart();
|
||||
return granularity.getIterable(new Interval(timeStart, timeStart.plus(1))).iterator().next().getStart();
|
||||
} else {
|
||||
return null;
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.BoundDimFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.FalseDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
||||
|
@ -52,7 +53,10 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
|
|||
@Override
|
||||
public DimFilter process(DimFilter filter)
|
||||
{
|
||||
if (filter instanceof AndDimFilter) {
|
||||
if (filter instanceof FalseDimFilter) {
|
||||
// we might sometimes come into here with just a false from optimizing impossible conditions
|
||||
return filter;
|
||||
} else if (filter instanceof AndDimFilter) {
|
||||
final List<DimFilter> children = getAndFilterChildren((AndDimFilter) filter);
|
||||
final DimFilter one = doSimplifyAnd(children);
|
||||
final DimFilter two = negate(doSimplifyOr(negateAll(children)));
|
||||
|
@ -130,15 +134,25 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
|
|||
// Group Bound filters by dimension, extractionFn, and comparator and compute a RangeSet for each one.
|
||||
final Map<BoundRefKey, List<BoundDimFilter>> bounds = new HashMap<>();
|
||||
|
||||
// all and/or filters have at least 1 child
|
||||
boolean allFalse = true;
|
||||
for (final DimFilter child : newChildren) {
|
||||
if (child instanceof BoundDimFilter) {
|
||||
final BoundDimFilter bound = (BoundDimFilter) child;
|
||||
final BoundRefKey boundRefKey = BoundRefKey.from(bound);
|
||||
final List<BoundDimFilter> filterList = bounds.computeIfAbsent(boundRefKey, k -> new ArrayList<>());
|
||||
filterList.add(bound);
|
||||
allFalse = false;
|
||||
} else {
|
||||
allFalse &= child instanceof FalseDimFilter;
|
||||
}
|
||||
}
|
||||
|
||||
// short circuit if can never be true
|
||||
if (allFalse) {
|
||||
return Filtration.matchNothing();
|
||||
}
|
||||
|
||||
// Try to simplify filters within each group.
|
||||
for (Map.Entry<BoundRefKey, List<BoundDimFilter>> entry : bounds.entrySet()) {
|
||||
final BoundRefKey boundRefKey = entry.getKey();
|
||||
|
|
|
@ -96,6 +96,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction;
|
|||
import org.apache.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.query.topn.DimensionTopNMetricSpec;
|
||||
import org.apache.druid.query.topn.InvertedTopNMetricSpec;
|
||||
import org.apache.druid.query.topn.NumericTopNMetricSpec;
|
||||
|
@ -5214,6 +5215,29 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithImpossibleTimeFilter() throws Exception
|
||||
{
|
||||
// this gets optimized into 'false'
|
||||
testQuery(
|
||||
"SELECT dim1, COUNT(*) FROM druid.foo\n"
|
||||
+ "WHERE FLOOR(__time TO DAY) = TIMESTAMP '2000-01-02 01:00:00'\n"
|
||||
+ "OR FLOOR(__time TO DAY) = TIMESTAMP '2000-01-02 02:00:00'\n"
|
||||
+ "GROUP BY 1",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of()))
|
||||
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
|
||||
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByOneColumnWithLiterallyFalseFilter() throws Exception
|
||||
{
|
||||
|
@ -17058,5 +17082,4 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
).build()
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue