mirror of https://github.com/apache/druid.git
add native filters for "(filter) is true" and "(filter) is false" (#15182)
* add native filters for "(filter) is true" and "(filter) is false" changes: * add IsTrueDimFilter, IsFalseDimFilter, and abstract IsBooleanDimFilter for native json filter implementations of `(filter) IS TRUE` and `(filter) IS FALSE` * add IsBooleanFilter for actual filtering logic for these filters, which ignore includeUnknown to always use matches with false for true and !matches with true for false * fix test incorrectly adjusted to wrong answer in #15058 * add tests for default value mode
This commit is contained in:
parent
c58b7f40ee
commit
061cfee224
|
@ -51,7 +51,9 @@ import java.util.Set;
|
|||
@JsonSubTypes.Type(name = "false", value = FalseDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "null", value = NullFilter.class),
|
||||
@JsonSubTypes.Type(name = "equals", value = EqualityFilter.class),
|
||||
@JsonSubTypes.Type(name = "range", value = RangeFilter.class)
|
||||
@JsonSubTypes.Type(name = "range", value = RangeFilter.class),
|
||||
@JsonSubTypes.Type(name = "isfalse", value = IsFalseDimFilter.class),
|
||||
@JsonSubTypes.Type(name = "istrue", value = IsTrueDimFilter.class)
|
||||
})
|
||||
public interface DimFilter extends Cacheable
|
||||
{
|
||||
|
|
|
@ -58,6 +58,8 @@ public class DimFilterUtils
|
|||
static final byte EQUALS_CACHE_ID = 0x13;
|
||||
static final byte RANGE_CACHE_ID = 0x14;
|
||||
|
||||
static final byte IS_FILTER_BOOLEAN_FILTER_CACHE_ID = 0x15;
|
||||
|
||||
|
||||
public static final byte STRING_SEPARATOR = (byte) 0xFF;
|
||||
|
||||
|
|
|
@ -0,0 +1,120 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.segment.filter.IsBooleanFilter;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Abstract SQL three-value logic wrapper for some child {@link DimFilter} to implement '{filter} IS TRUE' and
|
||||
* '{filter} IS FALSE'.
|
||||
*
|
||||
* @see IsTrueDimFilter - IS TRUE
|
||||
* @see IsFalseDimFilter - IS FALSE
|
||||
* @see IsBooleanFilter - actual filtering logic
|
||||
*/
|
||||
public abstract class IsBooleanDimFilter extends AbstractOptimizableDimFilter
|
||||
{
|
||||
private final DimFilter field;
|
||||
private final boolean isTrue;
|
||||
|
||||
public IsBooleanDimFilter(
|
||||
DimFilter field,
|
||||
boolean isTrue
|
||||
)
|
||||
{
|
||||
if (field == null) {
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build("IS %s operator requires a non-null filter for field", isTrue ? "TRUE" : "FALSE");
|
||||
}
|
||||
this.field = field;
|
||||
this.isTrue = isTrue;
|
||||
}
|
||||
|
||||
@JsonProperty("field")
|
||||
public DimFilter getField()
|
||||
{
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new CacheKeyBuilder(DimFilterUtils.IS_FILTER_BOOLEAN_FILTER_CACHE_ID).appendBoolean(isTrue)
|
||||
.appendCacheable(field)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new IsBooleanFilter(field.toFilter(), isTrue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RangeSet<String> getDimensionRangeSet(String dimension)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getRequiredColumns()
|
||||
{
|
||||
return field.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
IsBooleanDimFilter that = (IsBooleanDimFilter) o;
|
||||
|
||||
if (field != null ? !field.equals(that.field) : that.field != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return isTrue == that.isTrue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(field, isTrue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "(" + field + ") IS " + (isTrue ? "TRUE" : "FALSE");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
public class IsFalseDimFilter extends IsBooleanDimFilter
|
||||
{
|
||||
public static IsFalseDimFilter of(DimFilter field)
|
||||
{
|
||||
return new IsFalseDimFilter(field);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public IsFalseDimFilter(
|
||||
@JsonProperty("field") DimFilter field
|
||||
)
|
||||
{
|
||||
super(field, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter optimize()
|
||||
{
|
||||
return new IsFalseDimFilter(getField().optimize());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
public class IsTrueDimFilter extends IsBooleanDimFilter
|
||||
{
|
||||
public static IsTrueDimFilter of(DimFilter field)
|
||||
{
|
||||
return new IsTrueDimFilter(field);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public IsTrueDimFilter(
|
||||
@JsonProperty("field") DimFilter field
|
||||
)
|
||||
{
|
||||
super(field, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter optimize()
|
||||
{
|
||||
return new IsTrueDimFilter(getField().optimize());
|
||||
}
|
||||
}
|
|
@ -186,7 +186,7 @@ public class ExpressionFilter implements Filter
|
|||
return Arrays.stream(dResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((double) o));
|
||||
}
|
||||
}
|
||||
return (includeUnknown && eval.value() == null) || eval.asBoolean();
|
||||
return eval.asBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,210 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.filter;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.filter.vector.BaseVectorValueMatcher;
|
||||
import org.apache.druid.query.filter.vector.ReadableVectorMatch;
|
||||
import org.apache.druid.query.filter.vector.VectorMatch;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.index.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* SQL three-value logic wrapper for some child {@link Filter} to implement '{filter} IS TRUE' and
|
||||
* '{filter} IS FALSE'. Primarily useful when living beneath a {@link NotFilter} because this filter purposely ignores
|
||||
* the value of {@code includeUnknown} and so always correctly only returns values that definitely match or do not match
|
||||
* the filter to produce correct results for '{filter} IS NOT TRUE' and '{filter} IS NOT FALSE'. This filter is a
|
||||
* relatively thin wrapper, so should be relatively harmless if used without a 'NOT' filter.
|
||||
*
|
||||
* @see org.apache.druid.query.filter.IsBooleanDimFilter
|
||||
* @see org.apache.druid.query.filter.IsTrueDimFilter
|
||||
* @see org.apache.druid.query.filter.IsFalseDimFilter
|
||||
*/
|
||||
public class IsBooleanFilter implements Filter
|
||||
{
|
||||
private final Filter baseFilter;
|
||||
private final boolean isTrue;
|
||||
|
||||
public IsBooleanFilter(Filter baseFilter, boolean isTrue)
|
||||
{
|
||||
this.baseFilter = baseFilter;
|
||||
this.isTrue = isTrue;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
|
||||
{
|
||||
final BitmapColumnIndex baseIndex = baseFilter.getBitmapColumnIndex(selector);
|
||||
if (baseIndex != null && (isTrue || baseIndex.getIndexCapabilities().isInvertible())) {
|
||||
return new BitmapColumnIndex()
|
||||
{
|
||||
private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic();
|
||||
@Override
|
||||
public ColumnIndexCapabilities getIndexCapabilities()
|
||||
{
|
||||
return baseIndex.getIndexCapabilities();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double estimateSelectivity(int totalRows)
|
||||
{
|
||||
return 1. - baseFilter.estimateSelectivity(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
|
||||
{
|
||||
if (isTrue) {
|
||||
return baseIndex.computeBitmapResult(bitmapResultFactory, false);
|
||||
}
|
||||
return bitmapResultFactory.complement(
|
||||
baseIndex.computeBitmapResult(bitmapResultFactory, useThreeValueLogic),
|
||||
selector.getNumRows()
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||
{
|
||||
final ValueMatcher baseMatcher = baseFilter.makeMatcher(factory);
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic();
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
if (isTrue) {
|
||||
return baseMatcher.matches(false);
|
||||
}
|
||||
return !baseMatcher.matches(useThreeValueLogic);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("baseMatcher", baseMatcher);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory factory)
|
||||
{
|
||||
final VectorValueMatcher baseMatcher = baseFilter.makeVectorMatcher(factory);
|
||||
|
||||
return new BaseVectorValueMatcher(baseMatcher)
|
||||
{
|
||||
private final VectorMatch scratch = VectorMatch.wrap(new int[factory.getMaxVectorSize()]);
|
||||
private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic();
|
||||
|
||||
@Override
|
||||
public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown)
|
||||
{
|
||||
if (isTrue) {
|
||||
return baseMatcher.match(mask, false);
|
||||
}
|
||||
final ReadableVectorMatch baseMatch = baseMatcher.match(mask, useThreeValueLogic);
|
||||
|
||||
scratch.copyFrom(mask);
|
||||
scratch.removeAll(baseMatch);
|
||||
assert scratch.isValid(mask);
|
||||
return scratch;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorizeMatcher(ColumnInspector inspector)
|
||||
{
|
||||
return baseFilter.canVectorizeMatcher(inspector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getRequiredColumns()
|
||||
{
|
||||
return baseFilter.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsRequiredColumnRewrite()
|
||||
{
|
||||
return baseFilter.supportsRequiredColumnRewrite();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter rewriteRequiredColumns(Map<String, String> columnRewrites)
|
||||
{
|
||||
return new IsBooleanFilter(baseFilter.rewriteRequiredColumns(columnRewrites), isTrue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
|
||||
{
|
||||
return baseFilter.supportsSelectivityEstimation(columnSelector, indexSelector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return StringUtils.format("(%s) IS %s", baseFilter, isTrue ? "TRUE" : "FALSE");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
IsBooleanFilter isFilter = (IsBooleanFilter) o;
|
||||
return Objects.equals(baseFilter, isFilter.baseFilter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
// to return a different hash from baseFilter
|
||||
return Objects.hash(1, baseFilter, isTrue);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class IsBooleanDimFilterTest extends InitializedNullHandlingTest
|
||||
{
|
||||
@Test
|
||||
public void testSerde() throws JsonProcessingException
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
EqualityFilter baseFilter = new EqualityFilter("x", ColumnType.STRING, "hello", null);
|
||||
|
||||
IsTrueDimFilter trueFilter = IsTrueDimFilter.of(baseFilter);
|
||||
String s = mapper.writeValueAsString(trueFilter);
|
||||
Assert.assertEquals(trueFilter, mapper.readValue(s, IsTrueDimFilter.class));
|
||||
|
||||
IsFalseDimFilter falseFilter = IsFalseDimFilter.of(baseFilter);
|
||||
s = mapper.writeValueAsString(falseFilter);
|
||||
Assert.assertEquals(falseFilter, mapper.readValue(s, IsFalseDimFilter.class));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCacheKey()
|
||||
{
|
||||
EqualityFilter f1 = new EqualityFilter("x", ColumnType.STRING, "hello", null);
|
||||
EqualityFilter f1_2 = new EqualityFilter("x", ColumnType.STRING, "hello", null);
|
||||
EqualityFilter f2 = new EqualityFilter("x", ColumnType.STRING, "world", null);
|
||||
EqualityFilter f3 = new EqualityFilter("x", ColumnType.STRING, "hello", new FilterTuning(true, null, null));
|
||||
Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey());
|
||||
Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey()));
|
||||
Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidParameters()
|
||||
{
|
||||
Throwable t = Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> new IsTrueDimFilter(null)
|
||||
);
|
||||
Assert.assertEquals("IS TRUE operator requires a non-null filter for field", t.getMessage());
|
||||
t = Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> new IsFalseDimFilter(null)
|
||||
);
|
||||
Assert.assertEquals("IS FALSE operator requires a non-null filter for field", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_equals()
|
||||
{
|
||||
EqualsVerifier.forClass(IsTrueDimFilter.class).usingGetClass()
|
||||
.withNonnullFields("field")
|
||||
.withIgnoredFields("cachedOptimizedFilter")
|
||||
.verify();
|
||||
|
||||
EqualsVerifier.forClass(IsFalseDimFilter.class).usingGetClass()
|
||||
.withNonnullFields("field")
|
||||
.withIgnoredFields("cachedOptimizedFilter")
|
||||
.verify();
|
||||
}
|
||||
}
|
|
@ -37,6 +37,8 @@ import org.apache.druid.math.expr.ExprEval;
|
|||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.filter.EqualityFilter;
|
||||
import org.apache.druid.query.filter.FilterTuning;
|
||||
import org.apache.druid.query.filter.IsFalseDimFilter;
|
||||
import org.apache.druid.query.filter.IsTrueDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.segment.IndexBuilder;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
|
@ -255,6 +257,26 @@ public class EqualityFilterTests
|
|||
NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)),
|
||||
ImmutableList.of("0", "2", "4")
|
||||
);
|
||||
// "(s0 = 'a') is not true", same rows as "s0 <> 'a'", but also with null rows
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))),
|
||||
ImmutableList.of("0", "2", "3", "4")
|
||||
);
|
||||
// "(s0 = 'a') is true", equivalent to "s0 = 'a'"
|
||||
assertFilterMatches(
|
||||
IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)),
|
||||
ImmutableList.of("1", "5")
|
||||
);
|
||||
// "(s0 = 'a') is false", equivalent results to "s0 <> 'a'"
|
||||
assertFilterMatches(
|
||||
IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)),
|
||||
ImmutableList.of("0", "2", "4")
|
||||
);
|
||||
// "(s0 = 'a') is not false", same rows as "s0 = 'a'", but also with null rows
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))),
|
||||
ImmutableList.of("1", "3", "5")
|
||||
);
|
||||
|
||||
try {
|
||||
// make sure if 3vl is disabled with behave with 2vl
|
||||
|
@ -288,6 +310,28 @@ public class EqualityFilterTests
|
|||
NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "noexist", null)),
|
||||
ImmutableList.of("0", "1", "2", "3", "4", "5")
|
||||
);
|
||||
|
||||
// in default value mode, is true/is false are basically pointless since they have the same behavior as = and <>
|
||||
// "(s0 = 'a') is not true" equivalent to "s0 <> 'a'"
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))),
|
||||
ImmutableList.of("0", "2", "3", "4")
|
||||
);
|
||||
// "(s0 = 'a') is true", equivalent to "s0 = 'a'"
|
||||
assertFilterMatches(
|
||||
IsTrueDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)),
|
||||
ImmutableList.of("1", "5")
|
||||
);
|
||||
// "(s0 = 'a') is false" equivalent to "s0 <> 'a'"
|
||||
assertFilterMatches(
|
||||
IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)),
|
||||
ImmutableList.of("0", "2", "3", "4")
|
||||
);
|
||||
// "(s0 = 'a') is not false", equivalent to "s0 = 'a'"
|
||||
assertFilterMatches(
|
||||
NotDimFilter.of(IsFalseDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null))),
|
||||
ImmutableList.of("1", "5")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -45,6 +45,8 @@ import org.apache.druid.query.extraction.TimeFormatExtractionFn;
|
|||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.ExpressionDimFilter;
|
||||
import org.apache.druid.query.filter.IsFalseDimFilter;
|
||||
import org.apache.druid.query.filter.IsTrueDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.NullFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
@ -379,22 +381,47 @@ public class Expressions
|
|||
{
|
||||
final SqlKind kind = expression.getKind();
|
||||
|
||||
if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) {
|
||||
return toFilter(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
virtualColumnRegistry,
|
||||
Iterables.getOnlyElement(((RexCall) expression).getOperands())
|
||||
);
|
||||
} else if (kind == SqlKind.IS_FALSE || kind == SqlKind.IS_NOT_TRUE) {
|
||||
return new NotDimFilter(
|
||||
toFilter(
|
||||
if (kind == SqlKind.IS_TRUE
|
||||
|| kind == SqlKind.IS_NOT_TRUE
|
||||
|| kind == SqlKind.IS_FALSE
|
||||
|| kind == SqlKind.IS_NOT_FALSE) {
|
||||
if (NullHandling.useThreeValueLogic()) {
|
||||
final DimFilter baseFilter = toFilter(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
virtualColumnRegistry,
|
||||
Iterables.getOnlyElement(((RexCall) expression).getOperands())
|
||||
);
|
||||
|
||||
if (kind == SqlKind.IS_TRUE) {
|
||||
return IsTrueDimFilter.of(baseFilter);
|
||||
} else if (kind == SqlKind.IS_NOT_TRUE) {
|
||||
return NotDimFilter.of(IsTrueDimFilter.of(baseFilter));
|
||||
} else if (kind == SqlKind.IS_FALSE) {
|
||||
return IsFalseDimFilter.of(baseFilter);
|
||||
} else { // SqlKind.IS_NOT_FALSE
|
||||
return NotDimFilter.of(IsFalseDimFilter.of(baseFilter));
|
||||
}
|
||||
} else {
|
||||
// legacy behavior
|
||||
if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) {
|
||||
return toFilter(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
virtualColumnRegistry,
|
||||
Iterables.getOnlyElement(((RexCall) expression).getOperands())
|
||||
)
|
||||
);
|
||||
);
|
||||
} else { // SqlKind.IS_FALSE || SqlKind.IS_NOT_TRUE
|
||||
return new NotDimFilter(
|
||||
toFilter(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
virtualColumnRegistry,
|
||||
Iterables.getOnlyElement(((RexCall) expression).getOperands())
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
} else if (kind == SqlKind.CAST && expression.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
|
||||
// Calcite sometimes leaves errant, useless cast-to-booleans inside filters. Strip them and continue.
|
||||
return toFilter(
|
||||
|
@ -403,9 +430,7 @@ public class Expressions
|
|||
virtualColumnRegistry,
|
||||
Iterables.getOnlyElement(((RexCall) expression).getOperands())
|
||||
);
|
||||
} else if (kind == SqlKind.AND
|
||||
|| kind == SqlKind.OR
|
||||
|| kind == SqlKind.NOT) {
|
||||
} else if (kind == SqlKind.AND || kind == SqlKind.OR || kind == SqlKind.NOT) {
|
||||
final List<DimFilter> filters = new ArrayList<>();
|
||||
for (final RexNode rexNode : ((RexCall) expression).getOperands()) {
|
||||
final DimFilter nextFilter = toFilter(
|
||||
|
@ -424,8 +449,7 @@ public class Expressions
|
|||
return new AndDimFilter(filters);
|
||||
} else if (kind == SqlKind.OR) {
|
||||
return new OrDimFilter(filters);
|
||||
} else {
|
||||
assert kind == SqlKind.NOT;
|
||||
} else { // SqlKind.NOT
|
||||
return new NotDimFilter(Iterables.getOnlyElement(filters));
|
||||
}
|
||||
} else {
|
||||
|
@ -488,6 +512,11 @@ public class Expressions
|
|||
final SqlKind kind = rexNode.getKind();
|
||||
|
||||
if (kind == SqlKind.IS_TRUE || kind == SqlKind.IS_NOT_FALSE) {
|
||||
if (NullHandling.useThreeValueLogic()) {
|
||||
// use expression filter to get istrue or notfalse expressions for correct 3vl behavior
|
||||
return toExpressionLeafFilter(plannerContext, rowSignature, rexNode);
|
||||
}
|
||||
// legacy behavior
|
||||
return toSimpleLeafFilter(
|
||||
plannerContext,
|
||||
rowSignature,
|
||||
|
@ -495,6 +524,11 @@ public class Expressions
|
|||
Iterables.getOnlyElement(((RexCall) rexNode).getOperands())
|
||||
);
|
||||
} else if (kind == SqlKind.IS_FALSE || kind == SqlKind.IS_NOT_TRUE) {
|
||||
if (NullHandling.useThreeValueLogic()) {
|
||||
// use expression filter to get isfalse or nottrue expressions for correct 3vl behavior
|
||||
return toExpressionLeafFilter(plannerContext, rowSignature, rexNode);
|
||||
}
|
||||
// legacy behavior
|
||||
return new NotDimFilter(
|
||||
toSimpleLeafFilter(
|
||||
plannerContext,
|
||||
|
|
|
@ -23,6 +23,8 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.IsFalseDimFilter;
|
||||
import org.apache.druid.query.filter.IsTrueDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
||||
|
@ -89,6 +91,22 @@ public abstract class BottomUpTransform implements Function<Filtration, Filtrati
|
|||
} else {
|
||||
return checkedProcess(filter);
|
||||
}
|
||||
} else if (filter instanceof IsTrueDimFilter) {
|
||||
final DimFilter oldFilter = ((IsTrueDimFilter) filter).getField();
|
||||
final DimFilter newFilter = apply0(oldFilter);
|
||||
if (!oldFilter.equals(newFilter)) {
|
||||
return checkedProcess(new IsTrueDimFilter(newFilter));
|
||||
} else {
|
||||
return checkedProcess(filter);
|
||||
}
|
||||
} else if (filter instanceof IsFalseDimFilter) {
|
||||
final DimFilter oldFilter = ((IsFalseDimFilter) filter).getField();
|
||||
final DimFilter newFilter = apply0(oldFilter);
|
||||
if (!oldFilter.equals(newFilter)) {
|
||||
return checkedProcess(new IsFalseDimFilter(newFilter));
|
||||
} else {
|
||||
return checkedProcess(filter);
|
||||
}
|
||||
} else {
|
||||
return checkedProcess(filter);
|
||||
}
|
||||
|
|
|
@ -114,6 +114,7 @@ public class Filtration
|
|||
MoveTimeFiltersToIntervals.instance(),
|
||||
ConvertBoundsToSelectors.create(rowSignature),
|
||||
ConvertSelectorsToIns.create(rowSignature),
|
||||
RemoveRedundantIsTrue.instance(),
|
||||
MoveMarkerFiltersToIntervals.instance(),
|
||||
ValidateNoMarkerFiltersRemain.instance()
|
||||
)
|
||||
|
@ -136,7 +137,8 @@ public class Filtration
|
|||
ImmutableList.of(
|
||||
CombineAndSimplifyBounds.instance(),
|
||||
ConvertBoundsToSelectors.create(rowSignature),
|
||||
ConvertSelectorsToIns.create(rowSignature)
|
||||
ConvertSelectorsToIns.create(rowSignature),
|
||||
RemoveRedundantIsTrue.instance()
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.filtration;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import org.apache.druid.query.filter.AndDimFilter;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.IsTrueDimFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Similar to {@link BottomUpTransform} except only removes redundant IS TRUE filters that are not inside of a NOT
|
||||
* filter. The planner leaves behind stuff like `(x == y) IS TRUE` which is a pointless delegate when not living inside
|
||||
* of a not filter to enforce proper three-value logic
|
||||
*/
|
||||
public class RemoveRedundantIsTrue implements Function<Filtration, Filtration>
|
||||
{
|
||||
private static final RemoveRedundantIsTrue INSTANCE = new RemoveRedundantIsTrue();
|
||||
|
||||
public static RemoveRedundantIsTrue instance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filtration apply(Filtration filtration)
|
||||
{
|
||||
if (filtration.getDimFilter() != null) {
|
||||
final Filtration retVal = Filtration.create(apply0(filtration.getDimFilter()), filtration.getIntervals());
|
||||
return filtration.equals(retVal) ? retVal : apply(retVal);
|
||||
} else {
|
||||
return filtration;
|
||||
}
|
||||
}
|
||||
|
||||
private DimFilter apply0(final DimFilter filter)
|
||||
{
|
||||
// check for AND, OR to process their children, and unwrap any IS TRUE not living under a NOT, anything else we
|
||||
// leave alone
|
||||
if (filter instanceof AndDimFilter) {
|
||||
final List<DimFilter> oldFilters = ((AndDimFilter) filter).getFields();
|
||||
final List<DimFilter> newFilters = new ArrayList<>();
|
||||
for (DimFilter oldFilter : oldFilters) {
|
||||
final DimFilter newFilter = apply0(oldFilter);
|
||||
if (newFilter != null) {
|
||||
newFilters.add(newFilter);
|
||||
}
|
||||
}
|
||||
if (!newFilters.equals(oldFilters)) {
|
||||
return new AndDimFilter(newFilters);
|
||||
} else {
|
||||
return filter;
|
||||
}
|
||||
} else if (filter instanceof OrDimFilter) {
|
||||
final List<DimFilter> oldFilters = ((OrDimFilter) filter).getFields();
|
||||
final List<DimFilter> newFilters = new ArrayList<>();
|
||||
for (DimFilter oldFilter : oldFilters) {
|
||||
final DimFilter newFilter = apply0(oldFilter);
|
||||
if (newFilter != null) {
|
||||
newFilters.add(newFilter);
|
||||
}
|
||||
}
|
||||
if (!newFilters.equals(oldFilters)) {
|
||||
return new OrDimFilter(newFilters);
|
||||
} else {
|
||||
return filter;
|
||||
}
|
||||
} else if (filter instanceof IsTrueDimFilter) {
|
||||
final DimFilter oldFilter = ((IsTrueDimFilter) filter).getField();
|
||||
final DimFilter newFilter = apply0(oldFilter);
|
||||
if (!oldFilter.equals(newFilter)) {
|
||||
return newFilter;
|
||||
} else {
|
||||
return oldFilter;
|
||||
}
|
||||
} else {
|
||||
return filter;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -59,6 +59,7 @@ import org.apache.druid.query.filter.DimFilter;
|
|||
import org.apache.druid.query.filter.EqualityFilter;
|
||||
import org.apache.druid.query.filter.ExpressionDimFilter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.filter.IsTrueDimFilter;
|
||||
import org.apache.druid.query.filter.NotDimFilter;
|
||||
import org.apache.druid.query.filter.NullFilter;
|
||||
import org.apache.druid.query.filter.OrDimFilter;
|
||||
|
@ -365,6 +366,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
return new NotDimFilter(filter);
|
||||
}
|
||||
|
||||
public static IsTrueDimFilter istrue(DimFilter filter)
|
||||
{
|
||||
return new IsTrueDimFilter(filter);
|
||||
}
|
||||
|
||||
public static InDimFilter in(String dimension, Collection<String> values, ExtractionFn extractionFn)
|
||||
{
|
||||
return new InDimFilter(dimension, values, extractionFn);
|
||||
|
|
|
@ -6470,7 +6470,9 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
expressionFilter("(\"j0.unnest\" == \"m2\")"),
|
||||
and(
|
||||
isNull("j0.unnest"),
|
||||
not(expressionFilter("(\"j0.unnest\" == \"m2\")"))
|
||||
NullHandling.sqlCompatible()
|
||||
? not(istrue(expressionFilter("(\"j0.unnest\" == \"m2\")")))
|
||||
: not(expressionFilter("(\"j0.unnest\" == \"m2\")"))
|
||||
)
|
||||
))
|
||||
.legacy(false)
|
||||
|
|
|
@ -5607,7 +5607,14 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
or(
|
||||
isNull("__j0.a0"),
|
||||
not(
|
||||
or(
|
||||
NullHandling.sqlCompatible()
|
||||
? istrue(
|
||||
or(
|
||||
not(expressionFilter("\"__j0.d0\"")),
|
||||
notNull("__j0.d0")
|
||||
)
|
||||
)
|
||||
: or(
|
||||
not(expressionFilter("\"__j0.d0\"")),
|
||||
notNull("__j0.d0")
|
||||
)
|
||||
|
|
|
@ -3101,7 +3101,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
equality("dim2", "a", ColumnType.STRING),
|
||||
and(
|
||||
isNull("dim2"),
|
||||
not(equality("dim2", "a", ColumnType.STRING))
|
||||
NullHandling.sqlCompatible()
|
||||
? not(istrue(equality("dim2", "a", ColumnType.STRING)))
|
||||
: not(selector("dim2", "a"))
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -3109,12 +3111,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
|
||||
ImmutableList.of(
|
||||
NullHandling.replaceWithDefault()
|
||||
// Matches everything but "abc"
|
||||
? new Object[]{5L}
|
||||
: new Object[]{2L}
|
||||
// match only null values
|
||||
: new Object[]{4L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -4847,13 +4849,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new LongSumAggregatorFactory("a1", "cnt"),
|
||||
not(equality("dim1", "abc", ColumnType.STRING))
|
||||
NullHandling.sqlCompatible()
|
||||
? not(istrue(equality("dim1", "abc", ColumnType.STRING)))
|
||||
: not(selector("dim1", "abc"))
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
new LongSumAggregatorFactory("a2", "cnt"),
|
||||
NullHandling.replaceWithDefault()
|
||||
? selector("dim1", "a", new SubstringDimExtractionFn(0, 1))
|
||||
: expressionFilter("(substring(\"dim1\", 0, 1) == 'a')")
|
||||
NullHandling.sqlCompatible()
|
||||
? expressionFilter("(substring(\"dim1\", 0, 1) == 'a')")
|
||||
: selector("dim1", "a", new SubstringDimExtractionFn(0, 1))
|
||||
|
||||
),
|
||||
new FilteredAggregatorFactory(
|
||||
|
|
Loading…
Reference in New Issue