mirror of https://github.com/apache/druid.git
fix issues with multi-value string constant expressions (#12025)
* add specialized constant selector for multi-valued string constants
This commit is contained in:
parent
0565f0e6a1
commit
45be2be368
|
@ -0,0 +1,115 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import org.apache.druid.query.filter.ValueMatcher;
|
||||||
|
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import org.apache.druid.segment.data.IndexedInts;
|
||||||
|
import org.apache.druid.segment.data.RangeIndexedInts;
|
||||||
|
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
import org.apache.druid.segment.historical.HistoricalDimensionSelector;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class ConstantMultiValueDimensionSelector implements HistoricalDimensionSelector
|
||||||
|
{
|
||||||
|
private final List<String> values;
|
||||||
|
private final RangeIndexedInts row;
|
||||||
|
|
||||||
|
public ConstantMultiValueDimensionSelector(List<String> values)
|
||||||
|
{
|
||||||
|
this.values = values;
|
||||||
|
this.row = new RangeIndexedInts();
|
||||||
|
row.setSize(values.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("values", values);
|
||||||
|
inspector.visit("row", row);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Object getObject()
|
||||||
|
{
|
||||||
|
return defaultGetObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<?> classOfObject()
|
||||||
|
{
|
||||||
|
return Object.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return CARDINALITY_UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return values.get(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts getRow()
|
||||||
|
{
|
||||||
|
return row;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(@Nullable String value)
|
||||||
|
{
|
||||||
|
return BooleanValueMatcher.of(values.stream().anyMatch(v -> Objects.equals(value, v)));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return BooleanValueMatcher.of(values.stream().anyMatch(predicate::apply));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts getRow(int offset)
|
||||||
|
{
|
||||||
|
return row;
|
||||||
|
}
|
||||||
|
}
|
|
@ -34,6 +34,8 @@ import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelecto
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Selector for a string-typed column, either single- or multi-valued. This is named a "dimension" selector for legacy
|
* Selector for a string-typed column, either single- or multi-valued. This is named a "dimension" selector for legacy
|
||||||
|
@ -164,6 +166,38 @@ public interface DimensionSelector extends ColumnValueSelector<Object>, Dimensio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static DimensionSelector multiConstant(@Nullable final List<String> values)
|
||||||
|
{
|
||||||
|
// this method treats null, [], and [null] equivalently as null
|
||||||
|
if (values == null || values.isEmpty()) {
|
||||||
|
return NullDimensionSelectorHolder.NULL_DIMENSION_SELECTOR;
|
||||||
|
} else if (values.size() == 1) {
|
||||||
|
// the single value constant selector is more optimized than the multi-value constant selector because the latter
|
||||||
|
// does not report value cardinality, but otherwise behaves identically when used for grouping or selecting to a
|
||||||
|
// normal multi-value dimension selector (getObject on a row with a single value returns the object instead of
|
||||||
|
// the list)
|
||||||
|
return constant(values.get(0));
|
||||||
|
} else {
|
||||||
|
return new ConstantMultiValueDimensionSelector(values);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static DimensionSelector multiConstant(@Nullable final List<String> values, @Nullable final ExtractionFn extractionFn)
|
||||||
|
{
|
||||||
|
if (extractionFn == null) {
|
||||||
|
return multiConstant(values);
|
||||||
|
} else {
|
||||||
|
if (values == null) {
|
||||||
|
// the single value constant selector is more optimized than the multi-value constant selector because the
|
||||||
|
// latter does not report value cardinality, but otherwise behaves identically when used for grouping or
|
||||||
|
// selecting to a normal multi-value dimension selector (getObject on a row with a single value returns the
|
||||||
|
// object instead of the list)
|
||||||
|
return constant(extractionFn.apply(null));
|
||||||
|
}
|
||||||
|
return multiConstant(values.stream().map(extractionFn::apply).collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the given selector constantly returns null. This method could be used in the beginning of execution of
|
* Checks if the given selector constantly returns null. This method could be used in the beginning of execution of
|
||||||
* some queries and making some aggregations for heuristic shortcuts.
|
* some queries and making some aggregations for heuristic shortcuts.
|
||||||
|
|
|
@ -201,6 +201,10 @@ public class ExpressionSelectors
|
||||||
|
|
||||||
if (baseSelector instanceof ConstantExprEvalSelector) {
|
if (baseSelector instanceof ConstantExprEvalSelector) {
|
||||||
// Optimization for dimension selectors on constants.
|
// Optimization for dimension selectors on constants.
|
||||||
|
if (plan.is(ExpressionPlan.Trait.NON_SCALAR_OUTPUT)) {
|
||||||
|
final String[] value = baseSelector.getObject().asStringArray();
|
||||||
|
return DimensionSelector.multiConstant(value == null ? null : Arrays.asList(value), extractionFn);
|
||||||
|
}
|
||||||
return DimensionSelector.constant(baseSelector.getObject().asString(), extractionFn);
|
return DimensionSelector.constant(baseSelector.getObject().asString(), extractionFn);
|
||||||
} else if (baseSelector instanceof NilColumnValueSelector) {
|
} else if (baseSelector instanceof NilColumnValueSelector) {
|
||||||
// Optimization for null dimension selector.
|
// Optimization for null dimension selector.
|
||||||
|
|
|
@ -0,0 +1,157 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.druid.query.extraction.StringFormatExtractionFn;
|
||||||
|
import org.apache.druid.query.extraction.SubstringDimExtractionFn;
|
||||||
|
import org.apache.druid.segment.data.IndexedInts;
|
||||||
|
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
public class ConstantMultiValueDimensionSelectorTest extends InitializedNullHandlingTest
|
||||||
|
{
|
||||||
|
private final DimensionSelector NULL_SELECTOR = DimensionSelector.multiConstant(null);
|
||||||
|
private final DimensionSelector EMPTY_SELECTOR = DimensionSelector.multiConstant(Collections.emptyList());
|
||||||
|
private final DimensionSelector SINGLE_SELECTOR = DimensionSelector.multiConstant(ImmutableList.of("billy"));
|
||||||
|
private final DimensionSelector CONST_SELECTOR = DimensionSelector.multiConstant(ImmutableList.of("billy", "douglas"));
|
||||||
|
private final DimensionSelector NULL_EXTRACTION_SELECTOR = DimensionSelector.multiConstant(
|
||||||
|
null,
|
||||||
|
new StringFormatExtractionFn("billy")
|
||||||
|
);
|
||||||
|
private final DimensionSelector CONST_EXTRACTION_SELECTOR = DimensionSelector.multiConstant(
|
||||||
|
ImmutableList.of("billy", "douglas", "billy"),
|
||||||
|
new SubstringDimExtractionFn(0, 4)
|
||||||
|
);
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetRow()
|
||||||
|
{
|
||||||
|
IndexedInts row = NULL_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(1, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
|
||||||
|
row = EMPTY_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(1, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
|
||||||
|
row = SINGLE_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(1, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
|
||||||
|
row = CONST_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(2, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
Assert.assertEquals(1, row.get(1));
|
||||||
|
|
||||||
|
row = NULL_EXTRACTION_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(1, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
|
||||||
|
row = CONST_EXTRACTION_SELECTOR.getRow();
|
||||||
|
Assert.assertEquals(3, row.size());
|
||||||
|
Assert.assertEquals(0, row.get(0));
|
||||||
|
Assert.assertEquals(1, row.get(1));
|
||||||
|
Assert.assertEquals(2, row.get(2));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLookupName()
|
||||||
|
{
|
||||||
|
Assert.assertNull(NULL_SELECTOR.lookupName(0));
|
||||||
|
|
||||||
|
Assert.assertNull(EMPTY_SELECTOR.lookupName(0));
|
||||||
|
|
||||||
|
Assert.assertEquals("billy", CONST_SELECTOR.lookupName(0));
|
||||||
|
Assert.assertEquals("douglas", CONST_SELECTOR.lookupName(1));
|
||||||
|
|
||||||
|
Assert.assertEquals("billy", NULL_EXTRACTION_SELECTOR.lookupName(0));
|
||||||
|
|
||||||
|
Assert.assertEquals("bill", CONST_EXTRACTION_SELECTOR.lookupName(0));
|
||||||
|
Assert.assertEquals("doug", CONST_EXTRACTION_SELECTOR.lookupName(1));
|
||||||
|
Assert.assertEquals("bill", CONST_EXTRACTION_SELECTOR.lookupName(2));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetObject()
|
||||||
|
{
|
||||||
|
Assert.assertNull(NULL_SELECTOR.lookupName(0));
|
||||||
|
|
||||||
|
Assert.assertNull(EMPTY_SELECTOR.lookupName(0));
|
||||||
|
|
||||||
|
Assert.assertEquals("billy", SINGLE_SELECTOR.getObject());
|
||||||
|
Assert.assertEquals(ImmutableList.of("billy", "douglas"), CONST_SELECTOR.getObject());
|
||||||
|
|
||||||
|
Assert.assertEquals("billy", NULL_EXTRACTION_SELECTOR.getObject());
|
||||||
|
|
||||||
|
Assert.assertEquals(ImmutableList.of("bill", "doug", "bill"), CONST_EXTRACTION_SELECTOR.getObject());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCoverage()
|
||||||
|
{
|
||||||
|
Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, CONST_SELECTOR.getValueCardinality());
|
||||||
|
Assert.assertNull(CONST_SELECTOR.idLookup());
|
||||||
|
Assert.assertEquals(Object.class, CONST_SELECTOR.classOfObject());
|
||||||
|
Assert.assertTrue(CONST_SELECTOR.nameLookupPossibleInAdvance());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValueMatcher()
|
||||||
|
{
|
||||||
|
Assert.assertTrue(NULL_SELECTOR.makeValueMatcher((String) null).matches());
|
||||||
|
Assert.assertFalse(NULL_SELECTOR.makeValueMatcher("douglas").matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher((String) null).matches());
|
||||||
|
Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher("douglas").matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("billy").matches());
|
||||||
|
Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("douglas").matches());
|
||||||
|
Assert.assertFalse(CONST_SELECTOR.makeValueMatcher("debbie").matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches());
|
||||||
|
Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher((String) null).matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("bill").matches());
|
||||||
|
Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("doug").matches());
|
||||||
|
Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(NULL_SELECTOR.makeValueMatcher(Predicates.isNull()).matches());
|
||||||
|
Assert.assertFalse(NULL_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher(Predicates.equalTo(null)).matches());
|
||||||
|
Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher(Predicates.equalTo("douglas")).matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches());
|
||||||
|
Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("douglas")).matches());
|
||||||
|
Assert.assertFalse(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("debbie")).matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches());
|
||||||
|
Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo(null)).matches());
|
||||||
|
|
||||||
|
Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("bill")).matches());
|
||||||
|
Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("doug")).matches());
|
||||||
|
Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches());
|
||||||
|
}
|
||||||
|
}
|
|
@ -42,6 +42,8 @@ import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
|
import org.apache.druid.segment.ConstantDimensionSelector;
|
||||||
|
import org.apache.druid.segment.ConstantMultiValueDimensionSelector;
|
||||||
import org.apache.druid.segment.DimensionSelector;
|
import org.apache.druid.segment.DimensionSelector;
|
||||||
import org.apache.druid.segment.IdLookup;
|
import org.apache.druid.segment.IdLookup;
|
||||||
import org.apache.druid.segment.RowAdapters;
|
import org.apache.druid.segment.RowAdapters;
|
||||||
|
@ -828,4 +830,35 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
||||||
Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue());
|
Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue());
|
||||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConstantDimensionSelectors()
|
||||||
|
{
|
||||||
|
ExpressionVirtualColumn constant = new ExpressionVirtualColumn(
|
||||||
|
"constant",
|
||||||
|
Parser.parse("1 + 2", TestExprMacroTable.INSTANCE),
|
||||||
|
ColumnType.LONG
|
||||||
|
);
|
||||||
|
DimensionSelector constantSelector = constant.makeDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of("constant"),
|
||||||
|
COLUMN_SELECTOR_FACTORY
|
||||||
|
);
|
||||||
|
Assert.assertTrue(constantSelector instanceof ConstantDimensionSelector);
|
||||||
|
Assert.assertEquals("3", constantSelector.getObject());
|
||||||
|
|
||||||
|
|
||||||
|
ExpressionVirtualColumn multiConstant = new ExpressionVirtualColumn(
|
||||||
|
"multi",
|
||||||
|
Parser.parse("string_to_array('a,b,c', ',')", TestExprMacroTable.INSTANCE),
|
||||||
|
ColumnType.STRING
|
||||||
|
);
|
||||||
|
|
||||||
|
DimensionSelector multiConstantSelector = multiConstant.makeDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of("multiConstant"),
|
||||||
|
COLUMN_SELECTOR_FACTORY
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertTrue(multiConstantSelector instanceof ConstantMultiValueDimensionSelector);
|
||||||
|
Assert.assertEquals(ImmutableList.of("a", "b", "c"), multiConstantSelector.getObject());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -245,7 +245,7 @@ public class MultiValueStringOperatorConversions
|
||||||
|
|
||||||
public static class StringToMultiString extends StringToArrayOperatorConversion
|
public static class StringToMultiString extends StringToArrayOperatorConversion
|
||||||
{
|
{
|
||||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
public static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||||
.operatorBuilder("STRING_TO_MV")
|
.operatorBuilder("STRING_TO_MV")
|
||||||
.operandTypeChecker(
|
.operandTypeChecker(
|
||||||
OperandTypes.sequence(
|
OperandTypes.sequence(
|
||||||
|
|
|
@ -174,7 +174,13 @@ public class DruidRexExecutor implements RexExecutor
|
||||||
// complex constant is not reducible, so just leave it as an expression
|
// complex constant is not reducible, so just leave it as an expression
|
||||||
literal = constExp;
|
literal = constExp;
|
||||||
} else {
|
} else {
|
||||||
literal = rexBuilder.makeLiteral(exprResult.value(), constExp.getType(), true);
|
if (exprResult.isArray()) {
|
||||||
|
// just leave array expressions on multi-value strings alone, we're going to push them down into a virtual
|
||||||
|
// column selector anyway
|
||||||
|
literal = constExp;
|
||||||
|
} else {
|
||||||
|
literal = rexBuilder.makeLiteral(exprResult.value(), constExp.getType(), true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
reducedValues.add(literal);
|
reducedValues.add(literal);
|
||||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||||
import org.apache.druid.sql.calcite.expression.Expressions;
|
import org.apache.druid.sql.calcite.expression.Expressions;
|
||||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||||
|
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringOperatorConversions;
|
||||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||||
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
|
||||||
import org.apache.druid.sql.calcite.schema.NamedDruidSchema;
|
import org.apache.druid.sql.calcite.schema.NamedDruidSchema;
|
||||||
|
@ -182,4 +183,27 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultiValueStringNotReduced()
|
||||||
|
{
|
||||||
|
DruidRexExecutor rexy = new DruidRexExecutor(PLANNER_CONTEXT);
|
||||||
|
RexNode call = rexBuilder.makeCall(
|
||||||
|
MultiValueStringOperatorConversions.StringToMultiString.SQL_FUNCTION,
|
||||||
|
rexBuilder.makeLiteral("a,b,c"),
|
||||||
|
rexBuilder.makeLiteral(",")
|
||||||
|
);
|
||||||
|
List<RexNode> reduced = new ArrayList<>();
|
||||||
|
rexy.reduce(rexBuilder, ImmutableList.of(call), reduced);
|
||||||
|
Assert.assertEquals(1, reduced.size());
|
||||||
|
Assert.assertEquals(SqlKind.OTHER_FUNCTION, reduced.get(0).getKind());
|
||||||
|
Assert.assertEquals(
|
||||||
|
DruidExpression.fromExpression("string_to_array('a,b,c',',')"),
|
||||||
|
Expressions.toDruidExpression(
|
||||||
|
PLANNER_CONTEXT,
|
||||||
|
RowSignature.builder().build(),
|
||||||
|
reduced.get(0)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue