vectorized expressions and expression virtual columns (#10401)

* vectorized expression virtual columns

* cleanup

* fixes

* preserve float if explicitly specified

* oops

* null handling fixes, more tests

* what is an expression planner?

* better names

* remove unused method, add pi

* move vector processor builders into static methods

* reduce boilerplate

* oops

* more naming adjustments

* changes

* nullable

* missing hex

* more
This commit is contained in:
Clint Wylie 2020-09-23 13:56:38 -07:00 committed by GitHub
parent a60d034d01
commit 19c4b16640
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
94 changed files with 6487 additions and 630 deletions

View File

@ -47,6 +47,11 @@
<version>${jmh.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<version>4.0.2</version>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>

View File

@ -0,0 +1,219 @@
/*
* 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.benchmark;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.segment.vector.VectorCursor;
import org.apache.druid.segment.vector.VectorValueSelector;
import org.apache.druid.segment.virtual.ExpressionVectorSelectorsTest;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import javax.annotation.Nullable;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 3)
@Measurement(iterations = 5)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class ExpressionVectorSelectorBenchmark
{
static {
NullHandling.initializeForTests();
}
@Param({"1000000"})
private int rowsPerSegment;
@Param({"false", "true"})
private boolean vectorize;
@Param({
"long1 * long2",
"double1 * double3",
"float1 + float3",
"(long1 - long4) / double3",
"max(double3, double5)",
"min(double4, double1)",
"cos(float3)",
"sin(long4)",
"parse_long(string1)",
"parse_long(string1) * double3",
"parse_long(string5) * parse_long(string1)",
"parse_long(string5) * parse_long(string1) * double3"
})
private String expression;
private QueryableIndex index;
private Closer closer;
@Nullable
private ExprType outputType;
@Setup(Level.Trial)
public void setup()
{
this.closer = Closer.create();
final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("expression-testbench");
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.size(0)
.build();
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
this.index = closer.register(
segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment)
);
Expr parsed = Parser.parse(expression, ExprMacroTable.nil());
outputType = parsed.getOutputType(
new ColumnInspector()
{
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return QueryableIndexStorageAdapter.getColumnCapabilities(index, column);
}
}
);
checkSanity();
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
closer.close();
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void scan(Blackhole blackhole)
{
final VirtualColumns virtualColumns = VirtualColumns.create(
ImmutableList.of(
new ExpressionVirtualColumn(
"v",
expression,
ExprType.toValueType(outputType),
TestExprMacroTable.INSTANCE
)
)
);
if (vectorize) {
VectorCursor cursor = new QueryableIndexStorageAdapter(index).makeVectorCursor(
null,
index.getDataInterval(),
virtualColumns,
false,
512,
null
);
if (outputType.isNumeric()) {
VectorValueSelector selector = cursor.getColumnSelectorFactory().makeValueSelector("v");
if (outputType.equals(ExprType.DOUBLE)) {
while (!cursor.isDone()) {
blackhole.consume(selector.getDoubleVector());
blackhole.consume(selector.getNullVector());
cursor.advance();
}
} else {
while (!cursor.isDone()) {
blackhole.consume(selector.getLongVector());
blackhole.consume(selector.getNullVector());
cursor.advance();
}
}
closer.register(cursor);
}
} else {
Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
virtualColumns,
Granularities.ALL,
false,
null
);
int rowCount = cursors
.map(cursor -> {
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
int rows = 0;
while (!cursor.isDone()) {
blackhole.consume(selector.getObject());
rows++;
cursor.advance();
}
return rows;
}).accumulate(0, (acc, in) -> acc + in);
blackhole.consume(rowCount);
}
}
private void checkSanity()
{
ExpressionVectorSelectorsTest.sanityTestVectorizedExpressionSelectors(expression, outputType, index, closer, rowsPerSegment);
}
}

View File

@ -0,0 +1,298 @@
/*
* 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.benchmark.query;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* Benchmark that tests various SQL queries.
*/
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 3)
@Measurement(iterations = 5)
public class SqlExpressionBenchmark
{
private static final Logger log = new Logger(SqlExpressionBenchmark.class);
static {
NullHandling.initializeForTests();
Calcites.setSystemProperties();
}
private static final DruidProcessingConfig PROCESSING_CONFIG = new DruidProcessingConfig()
{
@Override
public int intermediateComputeSizeBytes()
{
return 512 * 1024 * 1024;
}
@Override
public int getNumMergeBuffers()
{
return 3;
}
@Override
public int getNumThreads()
{
return 1;
}
@Override
public boolean useParallelMergePoolConfigured()
{
return true;
}
@Override
public String getFormatString()
{
return "benchmarks-processing-%s";
}
};
private static final List<String> QUERIES = ImmutableList.of(
// ===========================
// non-expression reference queries
// ===========================
// 0: non-expression timeseries reference, 1 columns
"SELECT SUM(long1) FROM foo",
// 1: non-expression timeseries reference, 2 columns
"SELECT SUM(long1), SUM(long2) FROM foo",
// 2: non-expression timeseries reference, 3 columns
"SELECT SUM(long1), SUM(long4), SUM(double1) FROM foo",
// 3: non-expression timeseries reference, 4 columns
"SELECT SUM(long1), SUM(long4), SUM(double1), SUM(float3) FROM foo",
// 4: non-expression timeseries reference, 5 columns
"SELECT SUM(long1), SUM(long4), SUM(double1), SUM(float3), SUM(long5) FROM foo",
// 5: group by non-expr with 1 agg
"SELECT string2, SUM(long1) FROM foo GROUP BY 1 ORDER BY 2",
// 6: group by non-expr with 2 agg
"SELECT string2, SUM(long1), SUM(double3) FROM foo GROUP BY 1 ORDER BY 2",
// ===========================
// expressions
// ===========================
// 7: math op - 2 longs
"SELECT SUM(long1 * long2) FROM foo",
// 8: mixed math - 2 longs, 1 double
"SELECT SUM((long1 * long2) / double1) FROM foo",
// 9: mixed math - 2 longs, 1 double, 1 float
"SELECT SUM(float3 + ((long1 * long4)/double1)) FROM foo",
// 10: mixed math - 3 longs, 1 double, 1 float
"SELECT SUM(long5 - (float3 + ((long1 * long4)/double1))) FROM foo",
// 11: all same math op - 3 longs, 1 double, 1 float
"SELECT SUM(long5 * float3 * long1 * long4 * double1) FROM foo",
// 12: cos
"SELECT cos(double2) FROM foo",
// 13: unary negate
"SELECT SUM(-long4) FROM foo",
// 14: string long
"SELECT SUM(PARSE_LONG(string1)) FROM foo",
// 15: string longer
"SELECT SUM(PARSE_LONG(string3)) FROM foo",
// 16: time floor, non-expr col + reg agg
"SELECT TIME_FLOOR(__time, 'PT1H'), string2, SUM(double4) FROM foo GROUP BY 1,2 ORDER BY 3",
// 17: time floor, non-expr col + expr agg
"SELECT TIME_FLOOR(__time, 'PT1H'), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
// 18: time floor + non-expr agg (timeseries) (non-expression reference)
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1) FROM foo GROUP BY 1 ORDER BY 1",
// 19: time floor + expr agg (timeseries)
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 1",
// 20: time floor + non-expr agg (group by)
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1) FROM foo GROUP BY 1 ORDER BY 2",
// 21: time floor + expr agg (group by)
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 2",
// 22: time floor offset by 1 day + non-expr agg (group by)
"SELECT TIME_FLOOR(TIMESTAMPADD(DAY, -1, __time), 'PT1H'), SUM(long1) FROM foo GROUP BY 1 ORDER BY 1",
// 23: time floor offset by 1 day + expr agg (group by)
"SELECT TIME_FLOOR(TIMESTAMPADD(DAY, -1, __time), 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 1",
// 24: group by long expr with non-expr agg
"SELECT (long1 * long2), SUM(double1) FROM foo GROUP BY 1 ORDER BY 2",
// 25: group by non-expr with expr agg
"SELECT string2, SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 2"
);
@Param({"5000000"})
private int rowsPerSegment;
@Param({"false", "force"})
private String vectorize;
@Param({
// non-expression reference
"0",
"1",
"2",
"3",
"4",
"5",
"6",
// expressions
"7",
"8",
"9",
"10",
"11",
"12",
"13",
"14",
"15",
"16",
"17",
"18",
"19",
"20",
"21",
"22",
"23",
"24",
"25"
})
private String query;
@Nullable
private PlannerFactory plannerFactory;
private Closer closer = Closer.create();
@Setup(Level.Trial)
public void setup()
{
final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("expression-testbench");
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.size(0)
.build();
final PlannerConfig plannerConfig = new PlannerConfig();
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(
closer,
PROCESSING_CONFIG
);
final SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(
dataSegment,
index
);
closer.register(walker);
final SchemaPlus rootSchema =
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
plannerFactory = new PlannerFactory(
rootSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME
);
try {
SqlVectorizedExpressionSanityTest.sanityTestVectorizedSqlQueries(
plannerFactory,
QUERIES.get(Integer.parseInt(query))
);
}
catch (Throwable ignored) {
// the show must go on
}
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
closer.close();
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void querySql(Blackhole blackhole) throws Exception
{
final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
.createEscalatedAuthenticationResult();
try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) {
final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow);
}
}
}

View File

@ -27,6 +27,8 @@ import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -49,6 +51,31 @@ public interface ApplyFunction
*/
String name();
/**
* Check if an apply function can be 'vectorized', for a given {@link LambdaExpr} and set of {@link Expr} inputs.
* If this method returns true, {@link #asVectorProcessor} is expected to produce a {@link ExprVectorProcessor} which
* can evaluate values in batches to use with vectorized query engines.
*
* @see Expr#canVectorize(Expr.InputBindingTypes)
* @see Function#canVectorize(Expr.InputBindingTypes, List)
*/
default boolean canVectorize(Expr.InputBindingTypes inputTypes, Expr lambda, List<Expr> args)
{
return false;
}
/**
* Builds a 'vectorized' function expression processor, that can build vectorized processors for its input values
* using {@link Expr#buildVectorized}, for use in vectorized query engines.
*
* @see Expr#buildVectorized(Expr.VectorInputBindingTypes)
* @see Function#asVectorProcessor(Expr.VectorInputBindingTypes, List)
*/
default <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, Expr lambda, List<Expr> args)
{
throw new UOE("%s is not vectorized", name());
}
/**
* Apply {@link LambdaExpr} to argument list of {@link Expr} given a set of outer {@link Expr.ObjectBinding}. These
* outer bindings will be used to form the scope for the bindings used to evaluate the {@link LambdaExpr}, which use

View File

@ -20,6 +20,8 @@
package org.apache.druid.math.expr;
import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorComparisonProcessors;
import javax.annotation.Nullable;
import java.util.Objects;
@ -68,6 +70,18 @@ class BinLtExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.lessThan(inputTypes, left, right);
}
}
class BinLeqExpr extends BinaryEvalOpExprBase
@ -112,6 +126,18 @@ class BinLeqExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.lessThanOrEqual(inputTypes, left, right);
}
}
class BinGtExpr extends BinaryEvalOpExprBase
@ -156,6 +182,17 @@ class BinGtExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.greaterThan(inputTypes, left, right);
}
}
class BinGeqExpr extends BinaryEvalOpExprBase
@ -200,6 +237,18 @@ class BinGeqExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.greaterThanOrEqual(inputTypes, left, right);
}
}
class BinEqExpr extends BinaryEvalOpExprBase
@ -243,6 +292,18 @@ class BinEqExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.equal(inputTypes, left, right);
}
}
class BinNeqExpr extends BinaryEvalOpExprBase
@ -286,6 +347,18 @@ class BinNeqExpr extends BinaryEvalOpExprBase
}
return implicitCast;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorComparisonProcessors.notEqual(inputTypes, left, right);
}
}
class BinAndExpr extends BinaryOpExprBase

View File

@ -22,12 +22,14 @@ package org.apache.druid.math.expr;
import com.google.common.math.LongMath;
import com.google.common.primitives.Ints;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorMathProcessors;
import javax.annotation.Nullable;
// math operators live here
class BinPlusExpr extends BinaryEvalOpExprBase
final class BinPlusExpr extends BinaryEvalOpExprBase
{
BinPlusExpr(String op, Expr left, Expr right)
{
@ -43,24 +45,35 @@ class BinPlusExpr extends BinaryEvalOpExprBase
@Override
protected ExprEval evalString(@Nullable String left, @Nullable String right)
{
return ExprEval.of(NullHandling.nullToEmptyIfNeeded(left)
+ NullHandling.nullToEmptyIfNeeded(right));
return ExprEval.of(NullHandling.nullToEmptyIfNeeded(left) + NullHandling.nullToEmptyIfNeeded(right));
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return left + right;
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return left + right;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.plus(inputTypes, left, right);
}
}
class BinMinusExpr extends BinaryEvalOpExprBase
final class BinMinusExpr extends BinaryEvalOpExprBase
{
BinMinusExpr(String op, Expr left, Expr right)
{
@ -74,19 +87,31 @@ class BinMinusExpr extends BinaryEvalOpExprBase
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return left - right;
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return left - right;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.minus(inputTypes, left, right);
}
}
class BinMulExpr extends BinaryEvalOpExprBase
final class BinMulExpr extends BinaryEvalOpExprBase
{
BinMulExpr(String op, Expr left, Expr right)
{
@ -100,19 +125,31 @@ class BinMulExpr extends BinaryEvalOpExprBase
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return left * right;
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return left * right;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.multiply(inputTypes, left, right);
}
}
class BinDivExpr extends BinaryEvalOpExprBase
final class BinDivExpr extends BinaryEvalOpExprBase
{
BinDivExpr(String op, Expr left, Expr right)
{
@ -126,16 +163,28 @@ class BinDivExpr extends BinaryEvalOpExprBase
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return left / right;
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return left / right;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.divide(inputTypes, left, right);
}
}
class BinPowExpr extends BinaryEvalOpExprBase
@ -152,16 +201,28 @@ class BinPowExpr extends BinaryEvalOpExprBase
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return LongMath.pow(left, Ints.checkedCast(right));
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return Math.pow(left, right);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.power(inputTypes, left, right);
}
}
class BinModuloExpr extends BinaryEvalOpExprBase
@ -178,14 +239,26 @@ class BinModuloExpr extends BinaryEvalOpExprBase
}
@Override
protected final long evalLong(long left, long right)
protected long evalLong(long left, long right)
{
return left % right;
}
@Override
protected final double evalDouble(double left, double right)
protected double evalDouble(double left, double right)
{
return left % right;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.areNumeric(left, right) && inputTypes.canVectorize(left, right);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorMathProcessors.modulo(inputTypes, left, right);
}
}

View File

@ -23,6 +23,8 @@ import com.google.common.base.Preconditions;
import org.apache.commons.lang.StringEscapeUtils;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorProcessors;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -133,6 +135,18 @@ class LongExpr extends ConstantExpr
return ExprEval.ofLong(value);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorProcessors.constantLong(value, inputTypes.getMaxVectorSize());
}
@Override
public boolean equals(Object o)
{
@ -166,6 +180,18 @@ class NullLongExpr extends NullNumericConstantExpr
return ExprEval.ofLong(null);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorProcessors.constantLong(null, inputTypes.getMaxVectorSize());
}
@Override
public final int hashCode()
{
@ -236,6 +262,160 @@ class LongArrayExpr extends ConstantExpr
}
}
class DoubleExpr extends ConstantExpr
{
private final Double value;
DoubleExpr(Double value)
{
super(ExprType.DOUBLE);
this.value = Preconditions.checkNotNull(value, "value");
}
@Override
public Object getLiteralValue()
{
return value;
}
@Override
public String toString()
{
return String.valueOf(value);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDouble(value);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorProcessors.constantDouble(value, inputTypes.getMaxVectorSize());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DoubleExpr that = (DoubleExpr) o;
return Objects.equals(value, that.value);
}
@Override
public int hashCode()
{
return Objects.hash(value);
}
}
class NullDoubleExpr extends NullNumericConstantExpr
{
NullDoubleExpr()
{
super(ExprType.DOUBLE);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDouble(null);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorProcessors.constantDouble(null, inputTypes.getMaxVectorSize());
}
@Override
public final int hashCode()
{
return NullDoubleExpr.class.hashCode();
}
@Override
public final boolean equals(Object obj)
{
return obj instanceof NullDoubleExpr;
}
}
class DoubleArrayExpr extends ConstantExpr
{
private final Double[] value;
DoubleArrayExpr(Double[] value)
{
super(ExprType.DOUBLE_ARRAY);
this.value = Preconditions.checkNotNull(value, "value");
}
@Override
public Object getLiteralValue()
{
return value;
}
@Override
public String toString()
{
return Arrays.toString(value);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDoubleArray(value);
}
@Override
public String stringify()
{
if (value.length == 0) {
return "<DOUBLE>[]";
}
return StringUtils.format("<DOUBLE>%s", toString());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DoubleArrayExpr that = (DoubleArrayExpr) o;
return Arrays.equals(value, that.value);
}
@Override
public int hashCode()
{
return Arrays.hashCode(value);
}
}
class StringExpr extends ConstantExpr
{
@Nullable
@ -266,6 +446,18 @@ class StringExpr extends ConstantExpr
return ExprEval.of(value);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return VectorProcessors.constantString(value, inputTypes.getMaxVectorSize());
}
@Override
public String stringify()
{
@ -361,134 +553,3 @@ class StringArrayExpr extends ConstantExpr
return Arrays.hashCode(value);
}
}
class DoubleExpr extends ConstantExpr
{
private final Double value;
DoubleExpr(Double value)
{
super(ExprType.DOUBLE);
this.value = Preconditions.checkNotNull(value, "value");
}
@Override
public Object getLiteralValue()
{
return value;
}
@Override
public String toString()
{
return String.valueOf(value);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDouble(value);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DoubleExpr that = (DoubleExpr) o;
return Objects.equals(value, that.value);
}
@Override
public int hashCode()
{
return Objects.hash(value);
}
}
class NullDoubleExpr extends NullNumericConstantExpr
{
NullDoubleExpr()
{
super(ExprType.DOUBLE);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDouble(null);
}
@Override
public final int hashCode()
{
return NullDoubleExpr.class.hashCode();
}
@Override
public final boolean equals(Object obj)
{
return obj instanceof NullDoubleExpr;
}
}
class DoubleArrayExpr extends ConstantExpr
{
private final Double[] value;
DoubleArrayExpr(Double[] value)
{
super(ExprType.DOUBLE_ARRAY);
this.value = Preconditions.checkNotNull(value, "value");
}
@Override
public Object getLiteralValue()
{
return value;
}
@Override
public String toString()
{
return Arrays.toString(value);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return ExprEval.ofDoubleArray(value);
}
@Override
public String stringify()
{
if (value.length == 0) {
return "<DOUBLE>[]";
}
return StringUtils.format("<DOUBLE>%s", toString());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DoubleArrayExpr that = (DoubleArrayExpr) o;
return Arrays.equals(value, that.value);
}
@Override
public int hashCode()
{
return Arrays.hashCode(value);
}
}

View File

@ -24,9 +24,11 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -40,6 +42,7 @@ public interface Expr
{
String NULL_LITERAL = "null";
Joiner ARG_JOINER = Joiner.on(", ");
/**
* Indicates expression is a constant whose literal value can be extracted by {@link Expr#getLiteralValue()},
* making evaluating with arguments and bindings unecessary
@ -122,6 +125,7 @@ public interface Expr
*/
Expr visit(Shuttle shuttle);
/**
* Examine the usage of {@link IdentifierExpr} children of an {@link Expr}, constructing a {@link BindingAnalysis}
*/
@ -139,6 +143,25 @@ public interface Expr
return null;
}
/**
* Check if an expression can be 'vectorized', for a given set of inputs. If this method returns true,
* {@link #buildVectorized} is expected to produce a {@link ExprVectorProcessor} which can evaluate values in batches
* to use with vectorized query engines.
*/
default boolean canVectorize(InputBindingTypes inputTypes)
{
return false;
}
/**
* Builds a 'vectorized' expression processor, that can operate on batches of input values for use in vectorized
* query engines.
*/
default <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
throw Exprs.cannotVectorize(this);
}
/**
* Mechanism to supply input types for the bindings which will back {@link IdentifierExpr}, to use in the aid of
* inferring the output type of an expression with {@link #getOutputType}. A null value means that either the binding
@ -148,6 +171,63 @@ public interface Expr
{
@Nullable
ExprType getType(String name);
/**
* Check if all provided {@link Expr} can infer the output type as {@link ExprType#isNumeric} with a value of true.
*
* There must be at least one expression with a computable numeric output type for this method to return true.
*/
default boolean areNumeric(List<Expr> args)
{
boolean numeric = args.size() > 0;
for (Expr arg : args) {
ExprType argType = arg.getOutputType(this);
if (argType == null) {
numeric = false;
break;
}
numeric &= argType.isNumeric();
}
return numeric;
}
/**
* Check if all provided {@link Expr} can infer the output type as {@link ExprType#isNumeric} with a value of true.
*
* There must be at least one expression with a computable numeric output type for this method to return true.
*/
default boolean areNumeric(Expr... args)
{
return areNumeric(Arrays.asList(args));
}
/**
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingTypes)} to a value of true
*/
default boolean canVectorize(List<Expr> args)
{
boolean canVectorize = true;
for (Expr arg : args) {
canVectorize &= arg.canVectorize(this);
}
return canVectorize;
}
/**
* Check if every provided {@link Expr} computes {@link Expr#canVectorize(InputBindingTypes)} to a value of true
*/
default boolean canVectorize(Expr... args)
{
return canVectorize(Arrays.asList(args));
}
}
/**
* {@link InputBindingTypes} + vectorizations stuff for {@link #buildVectorized}
*/
interface VectorInputBindingTypes extends InputBindingTypes
{
int getMaxVectorSize();
}
/**
@ -162,6 +242,23 @@ public interface Expr
Object get(String name);
}
/**
* Mechanism to supply batches of input values to a {@link ExprVectorProcessor} for optimized processing. Mirrors
* the vectorized column selector interfaces, and includes {@link ExprType} information about all input bindings
* which exist
*/
interface VectorInputBinding extends VectorInputBindingTypes
{
<T> T[] getObjectVector(String name);
long[] getLongVector(String name);
double[] getDoubleVector(String name);
@Nullable
boolean[] getNullVector(String name);
int getCurrentVectorSize();
}
/**
* Mechanism to inspect an {@link Expr}, implementing a {@link Visitor} allows visiting all children of an
* {@link Expr}

View File

@ -121,6 +121,23 @@ public abstract class ExprEval<T>
return new StringExprEval(val == null ? null : String.valueOf(val));
}
@Nullable
public static Number computeNumber(@Nullable String value)
{
if (value == null) {
return null;
}
Number rv;
Long v = GuavaUtils.tryParseLong(value);
// Do NOT use ternary operator here, because it makes Java to convert Long to Double
if (v != null) {
rv = v;
} else {
rv = Doubles.tryParse(value);
}
return rv;
}
// Cached String values
private boolean stringValueCached = false;
@Nullable
@ -496,7 +513,7 @@ public abstract class ExprEval<T>
}
@Nullable
private Number computeNumber()
Number computeNumber()
{
if (value == null) {
return null;
@ -505,17 +522,8 @@ public abstract class ExprEval<T>
// Optimization for non-null case.
return numericVal;
}
Number rv;
Long v = GuavaUtils.tryParseLong(value);
// Do NOT use ternary operator here, because it makes Java to convert Long to Double
if (v != null) {
rv = v;
} else {
rv = Doubles.tryParse(value);
}
numericVal = rv;
return rv;
numericVal = computeNumber(value);
return numericVal;
}
@Override

View File

@ -73,10 +73,10 @@ public class ExprListenerImpl extends ExprBaseListener
int opCode = ((TerminalNode) ctx.getChild(0)).getSymbol().getType();
switch (opCode) {
case ExprParser.MINUS:
nodes.put(ctx, new UnaryMinusExpr((Expr) nodes.get(ctx.getChild(1))));
nodes.put(ctx, new UnaryMinusExpr(ctx.getChild(0).getText(), (Expr) nodes.get(ctx.getChild(1))));
break;
case ExprParser.NOT:
nodes.put(ctx, new UnaryNotExpr((Expr) nodes.get(ctx.getChild(1))));
nodes.put(ctx, new UnaryNotExpr(ctx.getChild(0).getText(), (Expr) nodes.get(ctx.getChild(1))));
break;
default:
throw new RE("Unrecognized unary operator %s", ctx.getChild(0).getText());

View File

@ -37,6 +37,7 @@ public enum ExprType
LONG_ARRAY,
STRING_ARRAY;
public boolean isNumeric()
{
return isNumeric(this);
@ -49,7 +50,7 @@ public enum ExprType
*
* @throws IllegalStateException
*/
public static ExprType fromValueType(@Nullable ValueType valueType)
public static ExprType fromValueTypeStrict(@Nullable ValueType valueType)
{
if (valueType == null) {
throw new IllegalStateException("Unsupported unknown value type");
@ -74,6 +75,58 @@ public enum ExprType
}
}
/**
* The expression system does not distinguish between {@link ValueType#FLOAT} and {@link ValueType#DOUBLE}, and
* cannot currently handle {@link ValueType#COMPLEX} inputs. This method will convert {@link ValueType#FLOAT} to
* {@link #DOUBLE}, or null if a null {@link ValueType#COMPLEX} is encountered.
*/
@Nullable
public static ExprType fromValueType(@Nullable ValueType valueType)
{
if (valueType == null) {
return null;
}
switch (valueType) {
case LONG:
return LONG;
case LONG_ARRAY:
return LONG_ARRAY;
case FLOAT:
case DOUBLE:
return DOUBLE;
case DOUBLE_ARRAY:
return DOUBLE_ARRAY;
case STRING:
return STRING;
case STRING_ARRAY:
return STRING_ARRAY;
case COMPLEX:
default:
return null;
}
}
public static ValueType toValueType(ExprType exprType)
{
switch (exprType) {
case LONG:
return ValueType.LONG;
case LONG_ARRAY:
return ValueType.LONG_ARRAY;
case DOUBLE:
return ValueType.DOUBLE;
case DOUBLE_ARRAY:
return ValueType.DOUBLE_ARRAY;
case STRING:
return ValueType.STRING;
case STRING_ARRAY:
return ValueType.STRING_ARRAY;
default:
throw new ISE("Unsupported expression type[%s]", exprType);
}
}
public static boolean isNumeric(ExprType type)
{
return LONG.equals(type) || DOUBLE.equals(type);

View File

@ -20,6 +20,7 @@
package org.apache.druid.math.expr;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.UOE;
import java.util.ArrayList;
import java.util.List;
@ -28,6 +29,21 @@ import java.util.Stack;
public class Exprs
{
public static UnsupportedOperationException cannotVectorize(Expr expr)
{
return new UOE("Unable to vectorize expression:[%s]", expr.stringify());
}
public static UnsupportedOperationException cannotVectorize(Function function)
{
return new UOE("Unable to vectorize function:[%s]", function.name());
}
public static UnsupportedOperationException cannotVectorize()
{
return new UOE("Unable to vectorize expression");
}
/**
* Decomposes any expr into a list of exprs that, if ANDed together, are equivalent to the input expr.
*

View File

@ -25,6 +25,11 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorMathProcessors;
import org.apache.druid.math.expr.vector.VectorProcessors;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
@ -112,6 +117,31 @@ public interface Function
@Nullable
ExprType getOutputType(Expr.InputBindingTypes inputTypes, List<Expr> args);
/**
* Check if a function can be 'vectorized', for a given set of {@link Expr} inputs. If this method returns true,
* {@link #asVectorProcessor} is expected to produce a {@link ExprVectorProcessor} which can evaluate values in
* batches to use with vectorized query engines.
*
* @see Expr#canVectorize(Expr.InputBindingTypes)
* @see ApplyFunction#canVectorize(Expr.InputBindingTypes, Expr, List)
*/
default boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return false;
}
/**
* Builds a 'vectorized' function expression processor, that can build vectorized processors for its input values
* using {@link Expr#buildVectorized}, for use in vectorized query engines.
*
* @see Expr#buildVectorized(Expr.VectorInputBindingTypes)
* @see ApplyFunction#asVectorProcessor(Expr.VectorInputBindingTypes, Expr, List)
*/
default <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
throw new UOE("%s is not vectorized", name());
}
/**
* Base class for a single variable input {@link Function} implementation
*/
@ -517,6 +547,25 @@ public interface Function
return ExprEval.of(retVal);
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return (args.size() == 1 || (args.get(1).isLiteral() && args.get(1).getLiteralValue() instanceof Number)) &&
inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
if (args.size() == 1 || args.get(1).isLiteral()) {
final int radix = args.size() == 1 ? 10 : ((Number) args.get(1).getLiteralValue()).intValue();
return VectorProcessors.parseLong(inputTypes, args.get(0), radix);
}
// only single argument and 2 argument where the radix is constant is currently implemented
// the canVectorize check should prevent this from happening, but explode just in case
throw Exprs.cannotVectorize(this);
}
}
class Pi implements Function
@ -549,6 +598,18 @@ public interface Function
{
return ExprType.DOUBLE;
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return true;
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorProcessors.constantDouble(PI, inputTypes.getMaxVectorSize());
}
}
class Abs extends UnivariateMathFunction
@ -615,6 +676,18 @@ public interface Function
{
return ExprEval.of(Math.atan(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.atan(inputTypes, args.get(0));
}
}
class Cbrt extends DoubleUnivariateMathFunction
@ -660,6 +733,18 @@ public interface Function
{
return ExprEval.of(Math.cos(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.cos(inputTypes, args.get(0));
}
}
class Cosh extends DoubleUnivariateMathFunction
@ -675,6 +760,18 @@ public interface Function
{
return ExprEval.of(Math.cosh(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.cosh(inputTypes, args.get(0));
}
}
class Cot extends DoubleUnivariateMathFunction
@ -690,6 +787,18 @@ public interface Function
{
return ExprEval.of(Math.cos(param) / Math.sin(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.cot(inputTypes, args.get(0));
}
}
class Div extends BivariateMathFunction
@ -711,6 +820,18 @@ public interface Function
{
return ExprEval.of((long) (x / y));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.divide(inputTypes, args.get(0), args.get(1));
}
}
class Exp extends DoubleUnivariateMathFunction
@ -964,6 +1085,18 @@ public interface Function
{
return ExprEval.of(Math.sin(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.sin(inputTypes, args.get(0));
}
}
class Sinh extends DoubleUnivariateMathFunction
@ -979,6 +1112,18 @@ public interface Function
{
return ExprEval.of(Math.sinh(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.sinh(inputTypes, args.get(0));
}
}
class Sqrt extends DoubleUnivariateMathFunction
@ -1009,6 +1154,18 @@ public interface Function
{
return ExprEval.of(Math.tan(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.tan(inputTypes, args.get(0));
}
}
class Tanh extends DoubleUnivariateMathFunction
@ -1024,6 +1181,18 @@ public interface Function
{
return ExprEval.of(Math.tanh(param));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.tanh(inputTypes, args.get(0));
}
}
class ToDegrees extends DoubleUnivariateMathFunction
@ -1150,6 +1319,18 @@ public interface Function
{
return ExprEval.of(Math.max(x, y));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.max(inputTypes, args.get(0), args.get(1));
}
}
class Min extends BivariateMathFunction
@ -1171,6 +1352,18 @@ public interface Function
{
return ExprEval.of(Math.min(x, y));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.min(inputTypes, args.get(0), args.get(1));
}
}
class NextAfter extends DoubleBivariateMathFunction
@ -1201,6 +1394,18 @@ public interface Function
{
return ExprEval.of(Math.pow(x, y));
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return inputTypes.areNumeric(args) && inputTypes.canVectorize(args);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return VectorMathProcessors.doublePower(inputTypes, args.get(0), args.get(1));
}
}
class Scalb extends BivariateFunction
@ -1295,6 +1500,21 @@ public interface Function
}
return null;
}
@Override
public boolean canVectorize(Expr.InputBindingTypes inputTypes, List<Expr> args)
{
return args.get(0).canVectorize(inputTypes) && args.get(1).isLiteral();
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingTypes inputTypes, List<Expr> args)
{
return CastToTypeVectorProcessor.castToType(
args.get(0).buildVectorized(inputTypes),
ExprType.valueOf(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString()))
);
}
}
class GreatestFunc extends ReduceFunction

View File

@ -22,6 +22,7 @@ package org.apache.druid.math.expr;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.List;
@ -76,6 +77,18 @@ class LambdaExpr implements Expr
return expr;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return expr.canVectorize(inputTypes);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return expr.buildVectorized(inputTypes);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
@ -170,6 +183,18 @@ class FunctionExpr implements Expr
return function.apply(args, bindings);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return function.canVectorize(inputTypes, args);
}
@Override
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingTypes inputTypes)
{
return function.asVectorProcessor(inputTypes, args);
}
@Override
public String stringify()
{
@ -288,6 +313,20 @@ class ApplyFunctionExpr implements Expr
return function.apply(lambdaExpr, argsExpr, bindings);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return function.canVectorize(inputTypes, lambdaExpr, argsExpr) &&
lambdaExpr.canVectorize(inputTypes) &&
argsExpr.stream().allMatch(expr -> expr.canVectorize(inputTypes));
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return function.asVectorProcessor(inputTypes, lambdaExpr, argsExpr);
}
@Override
public String stringify()
{

View File

@ -21,6 +21,11 @@ package org.apache.druid.math.expr;
import org.apache.commons.lang.StringEscapeUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprEvalDoubleVector;
import org.apache.druid.math.expr.vector.ExprEvalLongVector;
import org.apache.druid.math.expr.vector.ExprEvalStringVector;
import org.apache.druid.math.expr.vector.ExprEvalVector;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.Objects;
@ -32,8 +37,8 @@ import java.util.Objects;
*/
class IdentifierExpr implements Expr
{
private final String identifier;
private final String binding;
final String identifier;
final String binding;
/**
* Construct a identifier expression for a {@link LambdaExpr}, where the {@link #identifier} is equal to
@ -138,6 +143,53 @@ class IdentifierExpr implements Expr
return shuttle.visit(this);
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return inputTypes.getType(binding) != null;
}
@Override
public ExprVectorProcessor<?> buildVectorized(VectorInputBindingTypes inputTypes)
{
ExprType inputType = inputTypes.getType(binding);
if (inputType == null) {
throw Exprs.cannotVectorize(this);
}
switch (inputType) {
case LONG:
return new IdentifierVectorProcessor<long[]>(ExprType.LONG)
{
@Override
public ExprEvalVector<long[]> evalVector(VectorInputBinding bindings)
{
return new ExprEvalLongVector(bindings.getLongVector(binding), bindings.getNullVector(binding));
}
};
case DOUBLE:
return new IdentifierVectorProcessor<double[]>(ExprType.DOUBLE)
{
@Override
public ExprEvalVector<double[]> evalVector(VectorInputBinding bindings)
{
return new ExprEvalDoubleVector(bindings.getDoubleVector(binding), bindings.getNullVector(binding));
}
};
case STRING:
return new IdentifierVectorProcessor<String[]>(ExprType.STRING)
{
@Override
public ExprEvalVector<String[]> evalVector(VectorInputBinding bindings)
{
return new ExprEvalStringVector(bindings.getObjectVector(binding));
}
};
default:
throw Exprs.cannotVectorize(this);
}
}
@Override
public boolean equals(Object o)
{
@ -157,3 +209,20 @@ class IdentifierExpr implements Expr
return Objects.hash(identifier);
}
}
abstract class IdentifierVectorProcessor<T> implements ExprVectorProcessor<T>
{
private final ExprType outputType;
public IdentifierVectorProcessor(ExprType outputType)
{
this.outputType = outputType;
}
@Override
public ExprType getOutputType()
{
return outputType;
}
}

View File

@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.VectorMathProcessors;
import javax.annotation.Nullable;
import java.util.Objects;
@ -32,10 +34,12 @@ import java.util.Objects;
*/
abstract class UnaryExpr implements Expr
{
final String op;
final Expr expr;
UnaryExpr(Expr expr)
UnaryExpr(String op, Expr expr)
{
this.op = op;
this.expr = expr;
}
@ -91,19 +95,31 @@ abstract class UnaryExpr implements Expr
{
return Objects.hash(expr);
}
@Override
public String stringify()
{
return StringUtils.format("%s%s", op, expr.stringify());
}
@Override
public String toString()
{
return StringUtils.format("%s%s", op, expr);
}
}
class UnaryMinusExpr extends UnaryExpr
{
UnaryMinusExpr(Expr expr)
UnaryMinusExpr(String op, Expr expr)
{
super(expr);
super(op, expr);
}
@Override
UnaryExpr copy(Expr expr)
{
return new UnaryMinusExpr(expr);
return new UnaryMinusExpr(op, expr);
}
@Override
@ -123,29 +139,29 @@ class UnaryMinusExpr extends UnaryExpr
}
@Override
public String stringify()
public boolean canVectorize(InputBindingTypes inputTypes)
{
return StringUtils.format("-%s", expr.stringify());
return inputTypes.areNumeric(expr) && expr.canVectorize(inputTypes);
}
@Override
public String toString()
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
return StringUtils.format("-%s", expr);
return VectorMathProcessors.negate(inputTypes, expr);
}
}
class UnaryNotExpr extends UnaryExpr
{
UnaryNotExpr(Expr expr)
UnaryNotExpr(String op, Expr expr)
{
super(expr);
super(op, expr);
}
@Override
UnaryExpr copy(Expr expr)
{
return new UnaryNotExpr(expr);
return new UnaryNotExpr(op, expr);
}
@Override
@ -160,18 +176,6 @@ class UnaryNotExpr extends UnaryExpr
return ExprEval.of(!ret.asBoolean(), retType);
}
@Override
public String stringify()
{
return StringUtils.format("!%s", expr.stringify());
}
@Override
public String toString()
{
return StringUtils.format("!%s", expr);
}
@Nullable
@Override
public ExprType getOutputType(InputBindingTypes inputTypes)

View File

@ -0,0 +1,87 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
/**
* common machinery for processing two input operators and functions, which should always treat null inputs as null
* output, and are backed by a primitive values instead of an object values (and need to use the null vectors instead of
* checking the vector themselves for nulls)
*/
public abstract class BivariateFunctionVectorProcessor<TLeftInput, TRightInput, TOutput>
implements ExprVectorProcessor<TOutput>
{
final ExprVectorProcessor<TLeftInput> left;
final ExprVectorProcessor<TRightInput> right;
final int maxVectorSize;
final boolean[] outNulls;
final TOutput outValues;
protected BivariateFunctionVectorProcessor(
ExprVectorProcessor<TLeftInput> left,
ExprVectorProcessor<TRightInput> right,
int maxVectorSize,
TOutput outValues
)
{
this.left = left;
this.right = right;
this.maxVectorSize = maxVectorSize;
this.outNulls = new boolean[maxVectorSize];
this.outValues = outValues;
}
@Override
public final ExprEvalVector<TOutput> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<TLeftInput> lhs = left.evalVector(bindings);
final ExprEvalVector<TRightInput> rhs = right.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final boolean[] leftNulls = lhs.getNullVector();
final boolean[] rightNulls = rhs.getNullVector();
final boolean hasLeftNulls = leftNulls != null;
final boolean hasRightNulls = rightNulls != null;
final boolean hasNulls = hasLeftNulls || hasRightNulls;
final TLeftInput leftInput = lhs.values();
final TRightInput rightInput = rhs.values();
if (hasNulls) {
for (int i = 0; i < currentSize; i++) {
outNulls[i] = (hasLeftNulls && leftNulls[i]) || (hasRightNulls && rightNulls[i]);
if (!outNulls[i]) {
processIndex(leftInput, rightInput, i);
}
}
} else {
for (int i = 0; i < currentSize; i++) {
processIndex(leftInput, rightInput, i);
outNulls[i] = false;
}
}
return asEval();
}
abstract void processIndex(TLeftInput leftInput, TRightInput rightInput, int i);
abstract ExprEvalVector<TOutput> asEval();
}

View File

@ -0,0 +1,44 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
public final class CastToDoubleVectorProcessor extends CastToTypeVectorProcessor<double[]>
{
public CastToDoubleVectorProcessor(ExprVectorProcessor<?> delegate)
{
super(delegate);
}
@Override
public ExprEvalVector<double[]> evalVector(Expr.VectorInputBinding bindings)
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
return new ExprEvalDoubleVector(result.getDoubleVector(), result.getNullVector());
}
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
public final class CastToLongVectorProcessor extends CastToTypeVectorProcessor<long[]>
{
public CastToLongVectorProcessor(ExprVectorProcessor<?> delegate)
{
super(delegate);
}
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
return new ExprEvalLongVector(result.getLongVector(), result.getNullVector());
}
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor<String[]>
{
public CastToStringVectorProcessor(ExprVectorProcessor<?> delegate)
{
super(delegate);
}
@Override
public ExprEvalVector<String[]> evalVector(Expr.VectorInputBinding bindings)
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
return new ExprEvalStringVector(result.asObjectVector(ExprType.STRING));
}
@Override
public ExprType getOutputType()
{
return ExprType.STRING;
}
}

View File

@ -0,0 +1,56 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Exprs;
public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorProcessor<TOutput>
{
protected final ExprVectorProcessor<?> delegate;
protected CastToTypeVectorProcessor(ExprVectorProcessor<?> delegate)
{
this.delegate = delegate;
}
public static <T> ExprVectorProcessor<T> castToType(ExprVectorProcessor<?> delegate, ExprType type)
{
final ExprVectorProcessor<?> caster;
if (delegate.getOutputType() == type) {
caster = delegate;
} else {
switch (type) {
case STRING:
caster = new CastToStringVectorProcessor(delegate);
break;
case LONG:
caster = new CastToLongVectorProcessor(delegate);
break;
case DOUBLE:
caster = new CastToDoubleVectorProcessor(delegate);
break;
default:
throw Exprs.cannotVectorize();
}
}
return (ExprVectorProcessor<T>) caster;
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link UnivariateFunctionVectorProcessor} for processing (double[]) -> double[]
*/
public abstract class DoubleOutDoubleInFunctionVectorProcessor
extends UnivariateFunctionVectorProcessor<double[], double[]>
{
public DoubleOutDoubleInFunctionVectorProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
{
super(processor, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(double input);
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
@Override
final void processIndex(double[] input, int i)
{
outValues[i] = apply(input[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link BivariateFunctionVectorProcessor} for processing (double[], long[]) -> double[]
*/
public abstract class DoubleOutDoubleLongInFunctionVectorProcessor
extends BivariateFunctionVectorProcessor<double[], long[], double[]>
{
public DoubleOutDoubleLongInFunctionVectorProcessor(
ExprVectorProcessor<double[]> left,
ExprVectorProcessor<long[]> right,
int maxVectorSize
)
{
super(left, right, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(double left, long right);
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
@Override
final void processIndex(double[] leftInput, long[] rightInput, int i)
{
outValues[i] = apply(leftInput[i], rightInput[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link BivariateFunctionVectorProcessor} for processing (double[], double[]) -> double[]
*/
public abstract class DoubleOutDoublesInFunctionVectorProcessor
extends BivariateFunctionVectorProcessor<double[], double[], double[]>
{
public DoubleOutDoublesInFunctionVectorProcessor(
ExprVectorProcessor<double[]> left,
ExprVectorProcessor<double[]> right,
int maxVectorSize
)
{
super(left, right, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(double left, double right);
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
@Override
final void processIndex(double[] leftInput, double[] rightInput, int i)
{
outValues[i] = apply(leftInput[i], rightInput[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link BivariateFunctionVectorProcessor} for processing (long[], double[]) -> double[]
*/
public abstract class DoubleOutLongDoubleInFunctionVectorProcessor
extends BivariateFunctionVectorProcessor<long[], double[], double[]>
{
public DoubleOutLongDoubleInFunctionVectorProcessor(
ExprVectorProcessor<long[]> left,
ExprVectorProcessor<double[]> right,
int maxVectorSize
)
{
super(left, right, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(long left, double right);
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
@Override
final void processIndex(long[] leftInput, double[] rightInput, int i)
{
outValues[i] = apply(leftInput[i], rightInput[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link UnivariateFunctionVectorProcessor} for processing (long[]) -> double[]
*/
public abstract class DoubleOutLongInFunctionVectorProcessor
extends UnivariateFunctionVectorProcessor<long[], double[]>
{
public DoubleOutLongInFunctionVectorProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(processor, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(long input);
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
@Override
final void processIndex(long[] input, int i)
{
outValues[i] = apply(input[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link BivariateFunctionVectorProcessor} for processing (long[], long[]) -> double[]
*/
public abstract class DoubleOutLongsInFunctionVectorProcessor
extends BivariateFunctionVectorProcessor<long[], long[], double[]>
{
public DoubleOutLongsInFunctionVectorProcessor(
ExprVectorProcessor<long[]> left,
ExprVectorProcessor<long[]> right,
int maxVectorSize
)
{
super(left, right, maxVectorSize, new double[maxVectorSize]);
}
public abstract double apply(long left, long right);
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
@Override
final void processIndex(long[] leftInput, long[] rightInput, int i)
{
outValues[i] = apply(leftInput[i], rightInput[i]);
}
@Override
final ExprEvalVector<double[]> asEval()
{
return new ExprEvalDoubleVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.math.expr.vector;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.math.expr.ExprType;
import java.util.Arrays;
public final class ExprEvalDoubleVector extends ExprEvalVector<double[]>
{
public ExprEvalDoubleVector(double[] values, boolean[] nulls)
{
super(values, nulls);
}
@Override
public ExprType getType()
{
return ExprType.DOUBLE;
}
@Override
public double[] values()
{
return values;
}
@Override
public long[] getLongVector()
{
return Arrays.stream(values).mapToLong(d -> (long) d).toArray();
}
@Override
public double[] getDoubleVector()
{
return values;
}
@Override
public <E> E asObjectVector(ExprType type)
{
switch (type) {
case STRING:
String[] s = new String[values.length];
if (nulls != null) {
for (int i = 0; i < values.length; i++) {
s[i] = nulls[i] ? null : String.valueOf(values[i]);
}
}
return (E) s;
default:
throw new IAE("Cannot convert %s to %s object vector", getType(), type);
}
}
}

View File

@ -0,0 +1,69 @@
/*
* 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.math.expr.vector;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.math.expr.ExprType;
import javax.annotation.Nullable;
import java.util.Arrays;
public final class ExprEvalLongVector extends ExprEvalVector<long[]>
{
public ExprEvalLongVector(long[] values, @Nullable boolean[] nulls)
{
super(values, nulls);
}
@Override
public ExprType getType()
{
return ExprType.LONG;
}
@Override
public long[] getLongVector()
{
return values;
}
@Override
public double[] getDoubleVector()
{
return Arrays.stream(values).asDoubleStream().toArray();
}
@Override
public <E> E asObjectVector(ExprType type)
{
switch (type) {
case STRING:
String[] s = new String[values.length];
if (nulls != null) {
for (int i = 0; i < values.length; i++) {
s[i] = nulls[i] ? null : String.valueOf(values[i]);
}
}
return (E) s;
default:
throw new IAE("Cannot convert %s to %s object vector", getType(), type);
}
}
}

View File

@ -0,0 +1,109 @@
/*
* 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.math.expr.vector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprType;
import javax.annotation.Nullable;
public final class ExprEvalStringVector extends ExprEvalVector<String[]>
{
@Nullable
private long[] longs;
@Nullable
private double[] doubles;
@Nullable
private boolean[] numericNulls;
public ExprEvalStringVector(String[] values)
{
super(values, null);
}
private void computeNumbers()
{
if (longs == null) {
longs = new long[values.length];
doubles = new double[values.length];
numericNulls = new boolean[values.length];
for (int i = 0; i < values.length; i++) {
Number n = ExprEval.computeNumber(values[i]);
if (n != null) {
longs[i] = n.longValue();
doubles[i] = n.doubleValue();
numericNulls[i] = false;
} else {
longs[i] = 0L;
doubles[i] = 0.0;
numericNulls[i] = NullHandling.sqlCompatible();
}
}
}
}
@Nullable
@Override
public boolean[] getNullVector()
{
computeNumbers();
return numericNulls;
}
@Override
public ExprType getType()
{
return ExprType.STRING;
}
@Override
public long[] getLongVector()
{
computeNumbers();
return longs;
}
@Override
public double[] getDoubleVector()
{
computeNumbers();
return doubles;
}
@Override
public <E> E getObjectVector()
{
return (E) values;
}
@Override
public <E> E asObjectVector(ExprType type)
{
switch (type) {
case STRING:
return (E) values;
default:
throw new IAE("Cannot convert %s to %s object vector", getType(), type);
}
}
}

View File

@ -0,0 +1,79 @@
/*
* 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.math.expr.vector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.ExprType;
import javax.annotation.Nullable;
import java.lang.reflect.Array;
/**
* Result of {@link ExprVectorProcessor#evalVector} which wraps the actual evaluated results of the operation over the
* input vector(s). Methods to get actual results mirror vectorized value and object selectors.
*
* The generic parameter T should be the native java array type of the vector result (long[], String[], etc.)
*/
public abstract class ExprEvalVector<T>
{
final T values;
@Nullable
final boolean[] nulls;
public ExprEvalVector(T values, @Nullable boolean[] nulls)
{
this.values = values;
this.nulls = nulls;
}
public T values()
{
return values;
}
@Nullable
public Object get(int index)
{
if (nulls == null || NullHandling.replaceWithDefault() || !nulls[index]) {
return Array.get(values, index);
}
return null;
}
@Nullable
public boolean[] getNullVector()
{
return nulls;
}
public abstract ExprType getType();
public abstract long[] getLongVector();
public abstract double[] getDoubleVector();
public <E> E getObjectVector()
{
// non-primitives should implement this
throw new IllegalArgumentException("Object vector not available");
}
public abstract <E> E asObjectVector(ExprType type);
}

View File

@ -0,0 +1,34 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
/**
* Interface describing vectorized expression processors, which can be specialized using input type information to
* produce optimized expression evaluators, which can operate on batches of primitive data with minimal object overhead
*/
public interface ExprVectorProcessor<TOutput>
{
ExprEvalVector<TOutput> evalVector(Expr.VectorInputBinding bindings);
ExprType getOutputType();
}

View File

@ -0,0 +1,53 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link UnivariateFunctionVectorProcessor} for processing (long[]) -> long[]
*/
public abstract class LongOutLongInFunctionVectorProcessor extends UnivariateFunctionVectorProcessor<long[], long[]>
{
public LongOutLongInFunctionVectorProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(processor, maxVectorSize, new long[maxVectorSize]);
}
public abstract long apply(long input);
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
@Override
final void processIndex(long[] input, int i)
{
outValues[i] = apply(input[i]);
}
@Override
final ExprEvalVector<long[]> asEval()
{
return new ExprEvalLongVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link BivariateFunctionVectorProcessor} for processing (long[], long[]) -> long[]
*/
public abstract class LongOutLongsInFunctionVectorProcessor
extends BivariateFunctionVectorProcessor<long[], long[], long[]>
{
public LongOutLongsInFunctionVectorProcessor(
ExprVectorProcessor<long[]> left,
ExprVectorProcessor<long[]> right,
int maxVectorSize
)
{
super(left, right, maxVectorSize, new long[maxVectorSize]);
}
public abstract long apply(long left, long right);
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
@Override
final void processIndex(long[] leftInput, long[] rightInput, int i)
{
outValues[i] = apply(leftInput[i], rightInput[i]);
}
@Override
final ExprEvalVector<long[]> asEval()
{
return new ExprEvalLongVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,46 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.ExprType;
/**
* specialized {@link UnivariateFunctionVectorObjectProcessor} for processing (String[]) -> long[]
*/
public abstract class LongOutStringInFunctionVectorProcessor
extends UnivariateFunctionVectorObjectProcessor<String[], long[]>
{
public LongOutStringInFunctionVectorProcessor(ExprVectorProcessor<String[]> processor, int maxVectorSize)
{
super(processor, maxVectorSize, new long[maxVectorSize]);
}
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
@Override
public final ExprEvalVector<long[]> asEval()
{
return new ExprEvalLongVector(outValues, outNulls);
}
}

View File

@ -0,0 +1,65 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
/**
* common machinery for processing single input operators and functions, which are backed by an object value instead of
* a primitive value (so do not need to use the null vector, and instead can check the value vector itself for nulls)
*/
public abstract class UnivariateFunctionVectorObjectProcessor<TInput, TOutput> implements ExprVectorProcessor<TOutput>
{
final ExprVectorProcessor<TInput> processor;
final int maxVectorSize;
final boolean[] outNulls;
final TOutput outValues;
public UnivariateFunctionVectorObjectProcessor(
ExprVectorProcessor<TInput> processor,
int maxVectorSize,
TOutput outValues
)
{
this.processor = processor;
this.maxVectorSize = maxVectorSize;
this.outNulls = new boolean[maxVectorSize];
this.outValues = outValues;
}
@Override
public ExprEvalVector<TOutput> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<TInput> lhs = processor.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final TInput input = lhs.values();
for (int i = 0; i < currentSize; i++) {
processIndex(input, outValues, outNulls, i);
}
return asEval();
}
public abstract void processIndex(TInput input, TOutput output, boolean[] outputNulls, int i);
public abstract ExprEvalVector<TOutput> asEval();
}

View File

@ -0,0 +1,78 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Expr;
/**
* common machinery for processing single input operators and functions, which should always treat null input as null
* output, and are backed by a primitive value instead of an object value (and need to use the null vector instead of
* checking the vector itself for nulls)
*/
public abstract class UnivariateFunctionVectorProcessor<TInput, TOutput> implements ExprVectorProcessor<TOutput>
{
final ExprVectorProcessor<TInput> processor;
final int maxVectorSize;
final boolean[] outNulls;
final TOutput outValues;
public UnivariateFunctionVectorProcessor(
ExprVectorProcessor<TInput> processor,
int maxVectorSize,
TOutput outValues
)
{
this.processor = processor;
this.maxVectorSize = maxVectorSize;
this.outNulls = new boolean[maxVectorSize];
this.outValues = outValues;
}
@Override
public final ExprEvalVector<TOutput> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<TInput> lhs = processor.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final boolean[] inputNulls = lhs.getNullVector();
final boolean hasNulls = inputNulls != null;
final TInput input = lhs.values();
if (hasNulls) {
for (int i = 0; i < currentSize; i++) {
outNulls[i] = inputNulls[i];
if (!outNulls[i]) {
processIndex(input, i);
}
}
} else {
for (int i = 0; i < currentSize; i++) {
outNulls[i] = false;
processIndex(input, i);
}
}
return asEval();
}
abstract void processIndex(TInput input, int i);
abstract ExprEvalVector<TOutput> asEval();
}

View File

@ -0,0 +1,397 @@
/*
* 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.math.expr.vector;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
public class VectorComparisonProcessors
{
public static <T> ExprVectorProcessor<T> equal(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left == right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(left == right);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(left == right);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(left == right);
}
}
);
}
public static <T> ExprVectorProcessor<T> notEqual(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left != right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(left != right);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(left != right);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(left != right);
}
}
);
}
public static <T> ExprVectorProcessor<T> greaterThanOrEqual(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left >= right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(Double.compare(left, right) >= 0);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(Double.compare(left, right) >= 0);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(Double.compare(left, right) >= 0);
}
}
);
}
public static <T> ExprVectorProcessor<T> greaterThan(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left > right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(Double.compare(left, right) > 0);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(Double.compare(left, right) > 0);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(Double.compare(left, right) > 0);
}
}
);
}
public static <T> ExprVectorProcessor<T> lessThanOrEqual(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left <= right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(Double.compare(left, right) <= 0);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(Double.compare(left, right) <= 0);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(Double.compare(left, right) <= 0);
}
}
);
}
public static <T> ExprVectorProcessor<T> lessThan(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right
)
{
return VectorMathProcessors.makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Evals.asLong(left < right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Evals.asDouble(Double.compare(left, right) < 0);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Evals.asDouble(Double.compare(left, right) < 0);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Evals.asDouble(Double.compare(left, right) < 0);
}
}
);
}
private VectorComparisonProcessors()
{
// No instantiation
}
}

View File

@ -0,0 +1,879 @@
/*
* 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.math.expr.vector;
import com.google.common.math.LongMath;
import com.google.common.primitives.Ints;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Exprs;
import java.util.function.Supplier;
public class VectorMathProcessors
{
/**
* Make a 1 argument math processor with the following type rules
* long -> long
* double -> double
*/
public static <T> ExprVectorProcessor<T> makeMathProcessor(
Expr.VectorInputBindingTypes inputTypes,
Expr arg,
Supplier<LongOutLongInFunctionVectorProcessor> longOutLongInSupplier,
Supplier<DoubleOutDoubleInFunctionVectorProcessor> doubleOutDoubleInSupplier
)
{
final ExprType inputType = arg.getOutputType(inputTypes);
ExprVectorProcessor<?> processor = null;
if (ExprType.LONG.equals(inputType)) {
processor = longOutLongInSupplier.get();
} else if (ExprType.DOUBLE.equals(inputType)) {
processor = doubleOutDoubleInSupplier.get();
}
if (processor == null) {
throw Exprs.cannotVectorize();
}
return (ExprVectorProcessor<T>) processor;
}
/**
* Make a 1 argument math processor with the following type rules
* long -> double
* double -> double
*/
public static <T> ExprVectorProcessor<T> makeDoubleMathProcessor(
Expr.VectorInputBindingTypes inputTypes,
Expr arg,
Supplier<DoubleOutLongInFunctionVectorProcessor> doubleOutLongInSupplier,
Supplier<DoubleOutDoubleInFunctionVectorProcessor> doubleOutDoubleInSupplier
)
{
final ExprType inputType = arg.getOutputType(inputTypes);
ExprVectorProcessor<?> processor = null;
if (ExprType.LONG.equals(inputType)) {
processor = doubleOutLongInSupplier.get();
} else if (ExprType.DOUBLE.equals(inputType)) {
processor = doubleOutDoubleInSupplier.get();
}
if (processor == null) {
throw Exprs.cannotVectorize();
}
return (ExprVectorProcessor<T>) processor;
}
/**
* Make a 2 argument, math processor with the following type rules
* long, long -> long
* long, double -> double
* double, long -> double
* double, double -> double
*/
public static <T> ExprVectorProcessor<T> makeMathProcessor(
Expr.VectorInputBindingTypes inputTypes,
Expr left,
Expr right,
Supplier<LongOutLongsInFunctionVectorProcessor> longOutLongsInProcessor,
Supplier<DoubleOutLongDoubleInFunctionVectorProcessor> doubleOutLongDoubleInProcessor,
Supplier<DoubleOutDoubleLongInFunctionVectorProcessor> doubleOutDoubleLongInProcessor,
Supplier<DoubleOutDoublesInFunctionVectorProcessor> doubleOutDoublesInProcessor
)
{
final ExprType leftType = left.getOutputType(inputTypes);
final ExprType rightType = right.getOutputType(inputTypes);
ExprVectorProcessor<?> processor = null;
if (ExprType.LONG.equals(leftType)) {
if (ExprType.LONG.equals(rightType)) {
processor = longOutLongsInProcessor.get();
} else if (ExprType.DOUBLE.equals(rightType)) {
processor = doubleOutLongDoubleInProcessor.get();
}
} else if (ExprType.DOUBLE.equals(leftType)) {
if (ExprType.LONG.equals(rightType)) {
processor = doubleOutDoubleLongInProcessor.get();
} else if (ExprType.DOUBLE.equals(rightType)) {
processor = doubleOutDoublesInProcessor.get();
}
}
if (processor == null) {
throw Exprs.cannotVectorize();
}
return (ExprVectorProcessor<T>) processor;
}
public static <T> ExprVectorProcessor<T> plus(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return left + right;
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return (double) left + right;
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return left + (double) right;
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return left + right;
}
}
);
}
public static <T> ExprVectorProcessor<T> minus(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return left - right;
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return (double) left - right;
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return left - (double) right;
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return left - right;
}
}
);
}
public static <T> ExprVectorProcessor<T> multiply(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return left * right;
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return (double) left * right;
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return left * (double) right;
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return left * right;
}
}
);
}
public static <T> ExprVectorProcessor<T> divide(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return left / right;
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return (double) left / right;
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return left / (double) right;
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return left / right;
}
}
);
}
public static <T> ExprVectorProcessor<T> modulo(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return left % right;
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return (double) left % right;
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return left % (double) right;
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return left % right;
}
}
);
}
public static <T> ExprVectorProcessor<T> negate(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeMathProcessor(
inputTypes,
arg,
() -> new LongOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long input)
{
return -input;
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return -input;
}
}
);
}
public static <T> ExprVectorProcessor<T> power(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return LongMath.pow(left, Ints.checkedCast(right));
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Math.pow(left, right);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Math.pow(left, right);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Math.pow(left, right);
}
}
);
}
public static <T> ExprVectorProcessor<T> doublePower(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
BivariateFunctionVectorProcessor<?, ?, ?> processor = null;
if (ExprType.LONG.equals(left.getOutputType(inputTypes))) {
if (ExprType.LONG.equals(right.getOutputType(inputTypes))) {
processor = new DoubleOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, long right)
{
return Math.pow(left, right);
}
};
}
}
if (processor != null) {
return (ExprVectorProcessor<T>) processor;
}
return power(inputTypes, left, right);
}
public static <T> ExprVectorProcessor<T> max(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Math.max(left, right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Math.max(left, right);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Math.max(left, right);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Math.max(left, right);
}
}
);
}
public static <T> ExprVectorProcessor<T> min(Expr.VectorInputBindingTypes inputTypes, Expr left, Expr right)
{
return makeMathProcessor(
inputTypes,
left,
right,
() -> new LongOutLongsInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long left, long right)
{
return Math.min(left, right);
}
},
() -> new DoubleOutLongDoubleInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long left, double right)
{
return Math.min(left, right);
}
},
() -> new DoubleOutDoubleLongInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, long right)
{
return Math.min(left, right);
}
},
() -> new DoubleOutDoublesInFunctionVectorProcessor(
left.buildVectorized(inputTypes),
right.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double left, double right)
{
return Math.min(left, right);
}
}
);
}
public static <T> ExprVectorProcessor<T> atan(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.atan(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.atan(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> cos(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.cos(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.cos(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> cosh(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.cosh(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.cosh(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> cot(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.cos(input) / Math.sin(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.cos(input) / Math.sin(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> sin(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.sin(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.sin(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> sinh(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.sinh(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.sinh(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> tan(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.tan(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.tan(input);
}
}
);
}
public static <T> ExprVectorProcessor<T> tanh(Expr.VectorInputBindingTypes inputTypes, Expr arg)
{
return makeDoubleMathProcessor(
inputTypes,
arg,
() -> new DoubleOutLongInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(long input)
{
return Math.tanh(input);
}
},
() -> new DoubleOutDoubleInFunctionVectorProcessor(
arg.buildVectorized(inputTypes),
inputTypes.getMaxVectorSize()
)
{
@Override
public double apply(double input)
{
return Math.tanh(input);
}
}
);
}
private VectorMathProcessors()
{
// No instantiation
}
}

View File

@ -0,0 +1,142 @@
/*
* 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.math.expr.vector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import javax.annotation.Nullable;
import java.util.Arrays;
public class VectorProcessors
{
public static <T> ExprVectorProcessor<T> constantString(@Nullable String constant, int maxVectorSize)
{
final String[] strings = new String[maxVectorSize];
Arrays.fill(strings, constant);
final ExprEvalStringVector eval = new ExprEvalStringVector(strings);
return new ExprVectorProcessor<T>()
{
@Override
public ExprEvalVector<T> evalVector(Expr.VectorInputBinding bindings)
{
return (ExprEvalVector<T>) eval;
}
@Override
public ExprType getOutputType()
{
return ExprType.STRING;
}
};
}
public static <T> ExprVectorProcessor<T> constantDouble(@Nullable Double constant, int maxVectorSize)
{
final double[] doubles = new double[maxVectorSize];
final boolean[] nulls;
if (constant == null) {
nulls = new boolean[maxVectorSize];
Arrays.fill(nulls, true);
} else {
nulls = null;
Arrays.fill(doubles, constant);
}
final ExprEvalDoubleVector eval = new ExprEvalDoubleVector(doubles, nulls);
return new ExprVectorProcessor<T>()
{
@Override
public ExprEvalVector<T> evalVector(Expr.VectorInputBinding bindings)
{
return (ExprEvalVector<T>) eval;
}
@Override
public ExprType getOutputType()
{
return ExprType.DOUBLE;
}
};
}
public static <T> ExprVectorProcessor<T> constantLong(@Nullable Long constant, int maxVectorSize)
{
final long[] longs = new long[maxVectorSize];
final boolean[] nulls;
if (constant == null) {
nulls = new boolean[maxVectorSize];
Arrays.fill(nulls, true);
} else {
nulls = null;
Arrays.fill(longs, constant);
}
final ExprEvalLongVector eval = new ExprEvalLongVector(longs, nulls);
return new ExprVectorProcessor<T>()
{
@Override
public ExprEvalVector<T> evalVector(Expr.VectorInputBinding bindings)
{
return (ExprEvalVector<T>) eval;
}
@Override
public ExprType getOutputType()
{
return ExprType.LONG;
}
};
}
public static <T> ExprVectorProcessor<T> parseLong(Expr.VectorInputBindingTypes inputTypes, Expr arg, int radix)
{
final ExprVectorProcessor<?> processor = new LongOutStringInFunctionVectorProcessor(
CastToTypeVectorProcessor.castToType(arg.buildVectorized(inputTypes), ExprType.STRING),
inputTypes.getMaxVectorSize()
)
{
@Override
public void processIndex(String[] strings, long[] longs, boolean[] outputNulls, int i)
{
try {
final String input = strings[i];
if (radix == 16 && (input.startsWith("0x") || input.startsWith("0X"))) {
// Strip leading 0x from hex strings.
longs[i] = Long.parseLong(input.substring(2), radix);
} else {
longs[i] = Long.parseLong(input, radix);
}
outputNulls[i] = false;
}
catch (NumberFormatException e) {
longs[i] = 0L;
outputNulls[i] = NullHandling.sqlCompatible();
}
}
};
return (ExprVectorProcessor<T>) processor;
}
private VectorProcessors()
{
// No instantiation
}
}

View File

@ -120,13 +120,13 @@ public class ExprTest
@Test
public void testEqualsContractForUnaryNotExpr()
{
EqualsVerifier.forClass(UnaryNotExpr.class).usingGetClass().verify();
EqualsVerifier.forClass(UnaryNotExpr.class).withIgnoredFields("op").usingGetClass().verify();
}
@Test
public void testEqualsContractForUnaryMinusExpr()
{
EqualsVerifier.forClass(UnaryMinusExpr.class).usingGetClass().verify();
EqualsVerifier.forClass(UnaryMinusExpr.class).withIgnoredFields("op").usingGetClass().verify();
}
@Test

View File

@ -468,7 +468,7 @@ public class FunctionTest extends InitializedNullHandlingTest
{
// Same types
assertExpr("greatest(y, 0)", 2L);
assertExpr("greatest(34.0, z, 5.0, 767.0", 767.0);
assertExpr("greatest(34.0, z, 5.0, 767.0)", 767.0);
assertExpr("greatest('B', x, 'A')", "foo");
// Different types
@ -496,7 +496,7 @@ public class FunctionTest extends InitializedNullHandlingTest
{
// Same types
assertExpr("least(y, 0)", 0L);
assertExpr("least(34.0, z, 5.0, 767.0", 3.1);
assertExpr("least(34.0, z, 5.0, 767.0)", 3.1);
assertExpr("least('B', x, 'A')", "A");
// Different types

View File

@ -0,0 +1,459 @@
/*
* 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.math.expr;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.vector.ExprEvalVector;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
import java.util.function.BooleanSupplier;
import java.util.function.DoubleSupplier;
import java.util.function.LongSupplier;
import java.util.function.Supplier;
/**
* randomize inputs to various vector expressions and make sure the results match nonvectorized expressions
*
* this is not a replacement for correctness tests, but will ensure that vectorized and non-vectorized expression
* evaluation is at least self consistent...
*/
public class VectorExprSanityTest extends InitializedNullHandlingTest
{
private static final Logger log = new Logger(VectorExprSanityTest.class);
private static final int NUM_ITERATIONS = 10;
private static final int VECTOR_SIZE = 512;
final Map<String, ExprType> types = ImmutableMap.<String, ExprType>builder()
.put("l1", ExprType.LONG)
.put("l2", ExprType.LONG)
.put("d1", ExprType.DOUBLE)
.put("d2", ExprType.DOUBLE)
.put("s1", ExprType.STRING)
.put("s2", ExprType.STRING)
.build();
@Test
public void testUnaryOperators()
{
final String[] functions = new String[]{"-"};
final String[] templates = new String[]{"%sd1", "%sl1"};
testFunctions(types, templates, functions);
}
@Test
public void testBinaryOperators()
{
final String[] columns = new String[]{"d1", "d2", "l1", "l2", "1", "1.0"};
final String[][] templateInputs = makeTemplateArgs(columns, columns);
final String[] templates =
Arrays.stream(templateInputs)
.map(i -> StringUtils.format("%s %s %s", i[0], "%s", i[1]))
.toArray(String[]::new);
final String[] args = new String[]{"+", "-", "*", "/", "^", "%", ">", ">=", "<", "<=", "==", "!="};
testFunctions(types, templates, args);
}
@Test
public void testBinaryOperatorTrees()
{
final String[] columns = new String[]{"d1", "l1", "1", "1.0"};
final String[] columns2 = new String[]{"d2", "l2", "2", "2.0"};
final String[][] templateInputs = makeTemplateArgs(columns, columns2, columns);
final String[] templates =
Arrays.stream(templateInputs)
.map(i -> StringUtils.format("(%s %s %s) %s %s", i[0], "%s", i[1], "%s", i[2]))
.toArray(String[]::new);
final String[] ops = new String[]{"+", "-", "*", "/"};
final String[][] args = makeTemplateArgs(ops, ops);
testFunctions(types, templates, args);
}
@Test
public void testUnivariateFunctions()
{
final String[] functions = new String[]{"parse_long"};
final String[] templates = new String[]{"%s(s1)", "%s(l1)", "%s(d1)"};
testFunctions(types, templates, functions);
}
@Test
public void testUnivariateMathFunctions()
{
final String[] functions = new String[]{"atan", "cos", "cosh", "cot", "sin", "sinh", "tan", "tanh"};
final String[] templates = new String[]{"%s(l1)", "%s(d1)", "%s(pi())"};
testFunctions(types, templates, functions);
}
@Test
public void testBivariateMathFunctions()
{
final String[] functions = new String[]{"max", "min", "pow"};
final String[] templates = new String[]{"%s(d1, d2)", "%s(d1, l1)", "%s(l1, d1)", "%s(l1, l2)"};
testFunctions(types, templates, functions);
}
@Test
public void testCast()
{
final String[] columns = new String[]{"d1", "l1", "s1"};
final String[] castTo = new String[]{"'STRING'", "'LONG'", "'DOUBLE'"};
final String[][] args = makeTemplateArgs(columns, castTo);
final String[] templates = new String[]{"cast(%s, %s)"};
testFunctions(types, templates, args);
}
static void testFunctions(Map<String, ExprType> types, String[] templates, String[] args)
{
for (String template : templates) {
for (String arg : args) {
String expr = StringUtils.format(template, arg);
testExpression(expr, types);
}
}
}
static void testFunctions(Map<String, ExprType> types, String[] templates, String[][] argsArrays)
{
for (String template : templates) {
for (Object[] args : argsArrays) {
String expr = StringUtils.format(template, args);
testExpression(expr, types);
}
}
}
static void testExpression(String expr, Map<String, ExprType> types)
{
log.debug("[%s]", expr);
Expr parsed = Parser.parse(expr, ExprMacroTable.nil());
NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> bindings;
for (int iterations = 0; iterations < NUM_ITERATIONS; iterations++) {
bindings = makeRandomizedBindings(VECTOR_SIZE, types);
testExpressionWithBindings(expr, parsed, bindings);
}
bindings = makeSequentialBinding(VECTOR_SIZE, types);
testExpressionWithBindings(expr, parsed, bindings);
}
private static void testExpressionWithBindings(
String expr,
Expr parsed,
NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> bindings
)
{
Assert.assertTrue(StringUtils.format("Cannot vectorize %s", expr), parsed.canVectorize(bindings.rhs));
ExprType outputType = parsed.getOutputType(bindings.rhs);
ExprEvalVector<?> vectorEval = parsed.buildVectorized(bindings.rhs).evalVector(bindings.rhs);
Assert.assertEquals(outputType, vectorEval.getType());
for (int i = 0; i < VECTOR_SIZE; i++) {
ExprEval<?> eval = parsed.eval(bindings.lhs[i]);
if (!eval.isNumericNull()) {
Assert.assertEquals(outputType, eval.type());
}
Assert.assertEquals(
StringUtils.format("Values do not match for row %s for expression %s", i, expr),
eval.value(),
vectorEval.get(i)
);
}
}
static NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> makeRandomizedBindings(
int vectorSize,
Map<String, ExprType> types
)
{
final ThreadLocalRandom r = ThreadLocalRandom.current();
return makeBindings(
vectorSize,
types,
() -> r.nextLong(Integer.MAX_VALUE - 1),
r::nextDouble,
r::nextBoolean,
() -> String.valueOf(r.nextInt())
);
}
static NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> makeSequentialBinding(
int vectorSize,
Map<String, ExprType> types
)
{
return makeBindings(
vectorSize,
types,
new LongSupplier()
{
int counter = 1;
@Override
public long getAsLong()
{
return counter++;
}
},
new DoubleSupplier()
{
int counter = 1;
@Override
public double getAsDouble()
{
return counter++;
}
},
() -> ThreadLocalRandom.current().nextBoolean(),
new Supplier<String>()
{
int counter = 1;
@Override
public String get()
{
return String.valueOf(counter++);
}
}
);
}
static NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> makeBindings(
int vectorSize,
Map<String, ExprType> types,
LongSupplier longsFn,
DoubleSupplier doublesFn,
BooleanSupplier nullsFn,
Supplier<String> stringFn
)
{
SettableVectorInputBinding vectorBinding = new SettableVectorInputBinding(vectorSize);
SettableObjectBinding[] objectBindings = new SettableObjectBinding[vectorSize];
final boolean hasNulls = NullHandling.sqlCompatible();
for (Map.Entry<String, ExprType> entry : types.entrySet()) {
boolean[] nulls = new boolean[vectorSize];
switch (entry.getValue()) {
case LONG:
long[] longs = new long[vectorSize];
for (int i = 0; i < vectorSize; i++) {
nulls[i] = hasNulls && nullsFn.getAsBoolean();
longs[i] = nulls[i] ? 0L : longsFn.getAsLong();
if (objectBindings[i] == null) {
objectBindings[i] = new SettableObjectBinding();
}
objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : longs[i]);
}
if (hasNulls) {
vectorBinding.addLong(entry.getKey(), longs, nulls);
} else {
vectorBinding.addLong(entry.getKey(), longs);
}
break;
case DOUBLE:
double[] doubles = new double[vectorSize];
for (int i = 0; i < vectorSize; i++) {
nulls[i] = hasNulls && nullsFn.getAsBoolean();
doubles[i] = nulls[i] ? 0.0 : doublesFn.getAsDouble();
if (objectBindings[i] == null) {
objectBindings[i] = new SettableObjectBinding();
}
objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : doubles[i]);
}
if (hasNulls) {
vectorBinding.addDouble(entry.getKey(), doubles, nulls);
} else {
vectorBinding.addDouble(entry.getKey(), doubles);
}
break;
case STRING:
String[] strings = new String[vectorSize];
for (int i = 0; i < vectorSize; i++) {
nulls[i] = hasNulls && nullsFn.getAsBoolean();
strings[i] = nulls[i] ? null : String.valueOf(stringFn.get());
if (objectBindings[i] == null) {
objectBindings[i] = new SettableObjectBinding();
}
objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : strings[i]);
}
vectorBinding.addString(entry.getKey(), strings);
break;
}
}
return new NonnullPair<>(objectBindings, vectorBinding);
}
static String[][] makeTemplateArgs(String[] arg1, String[] arg2)
{
return Arrays.stream(arg1)
.flatMap(a1 -> Arrays.stream(arg2).map(a2 -> new String[]{a1, a2}))
.toArray(String[][]::new);
}
static String[][] makeTemplateArgs(String[] arg1, String[] arg2, String[] arg3)
{
return Arrays.stream(arg1)
.flatMap(a1 ->
Arrays.stream(arg2).flatMap(a2 -> Arrays.stream(arg3).map(a3 -> new String[]{a1, a2, a3}))
)
.toArray(String[][]::new);
}
static class SettableObjectBinding implements Expr.ObjectBinding
{
private final Map<String, Object> bindings;
SettableObjectBinding()
{
this.bindings = new HashMap<>();
}
@Nullable
@Override
public Object get(String name)
{
return bindings.get(name);
}
public SettableObjectBinding withBinding(String name, @Nullable Object value)
{
bindings.put(name, value);
return this;
}
}
static class SettableVectorInputBinding implements Expr.VectorInputBinding
{
private final Map<String, boolean[]> nulls;
private final Map<String, long[]> longs;
private final Map<String, double[]> doubles;
private final Map<String, Object[]> objects;
private final Map<String, ExprType> types;
private final int vectorSize;
SettableVectorInputBinding(int vectorSize)
{
this.nulls = new HashMap<>();
this.longs = new HashMap<>();
this.doubles = new HashMap<>();
this.objects = new HashMap<>();
this.types = new HashMap<>();
this.vectorSize = vectorSize;
}
public SettableVectorInputBinding addBinding(String name, ExprType type, boolean[] nulls)
{
this.nulls.put(name, nulls);
this.types.put(name, type);
return this;
}
public SettableVectorInputBinding addLong(String name, long[] longs)
{
return addLong(name, longs, new boolean[longs.length]);
}
public SettableVectorInputBinding addLong(String name, long[] longs, boolean[] nulls)
{
assert longs.length == vectorSize;
this.longs.put(name, longs);
return addBinding(name, ExprType.LONG, nulls);
}
public SettableVectorInputBinding addDouble(String name, double[] doubles)
{
return addDouble(name, doubles, new boolean[doubles.length]);
}
public SettableVectorInputBinding addDouble(String name, double[] doubles, boolean[] nulls)
{
assert doubles.length == vectorSize;
this.doubles.put(name, doubles);
return addBinding(name, ExprType.DOUBLE, nulls);
}
public SettableVectorInputBinding addString(String name, String[] strings)
{
assert strings.length == vectorSize;
this.objects.put(name, strings);
return addBinding(name, ExprType.STRING, new boolean[strings.length]);
}
@Override
public <T> T[] getObjectVector(String name)
{
return (T[]) objects.get(name);
}
@Override
public ExprType getType(String name)
{
return types.get(name);
}
@Override
public long[] getLongVector(String name)
{
return longs.get(name);
}
@Override
public double[] getDoubleVector(String name)
{
return doubles.get(name);
}
@Nullable
@Override
public boolean[] getNullVector(String name)
{
return nulls.get(name);
}
@Override
public int getMaxVectorSize()
{
return vectorSize;
}
@Override
public int getCurrentVectorSize()
{
return vectorSize;
}
}
}

View File

@ -210,9 +210,6 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
@Test
public void testBloomFilterVirtualColumnNumber() throws Exception
{
// Cannot vectorize due to expression virtual columns.
cannotVectorize();
BloomKFilter filter = new BloomKFilter(1500);
filter.addFloat(20.2f);
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);

View File

@ -19,18 +19,25 @@
package org.apache.druid.query.aggregation;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DoubleColumnSelector;
import org.apache.druid.segment.FloatColumnSelector;
import org.apache.druid.segment.LongColumnSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import org.apache.druid.segment.virtual.ExpressionSelectors;
import org.apache.druid.segment.virtual.ExpressionVectorSelectors;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -191,7 +198,7 @@ public class AggregatorUtil
)
{
if ((fieldName == null) == (fieldExpression == null)) {
throw new IllegalArgumentException("Only one of fieldName and fieldExpression should be non-null");
throw new IllegalArgumentException("Only one of fieldName or expression should be non-null");
}
if (fieldName != null) {
return metricFactory.makeColumnValueSelector(fieldName);
@ -310,4 +317,40 @@ public class AggregatorUtil
return new ExpressionDoubleColumnSelector();
}
}
public static boolean canVectorize(
ColumnInspector columnInspector,
@Nullable String fieldName,
@Nullable String expression,
Supplier<Expr> fieldExpression
)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return capabilities == null || ValueType.isNumeric(capabilities.getType());
}
if (expression != null) {
return fieldExpression.get().canVectorize(columnInspector);
}
return false;
}
/**
* Make a {@link VectorValueSelector} for primitive numeric or expression virtual column inputs.
*/
public static VectorValueSelector makeVectorValueSelector(
VectorColumnSelectorFactory columnSelectorFactory,
@Nullable String fieldName,
@Nullable String expression,
Supplier<Expr> fieldExpression
)
{
if ((fieldName == null) == (expression == null)) {
throw new IllegalArgumentException("Only one of fieldName or expression should be non-null");
}
if (expression != null) {
return ExpressionVectorSelectors.makeVectorValueSelector(columnSelectorFactory, fieldExpression.get());
}
return columnSelectorFactory.makeValueSelector(fieldName);
}
}

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class DoubleMaxAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleMaxBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class DoubleMaxAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleMaxVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class DoubleMinAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleMinBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class DoubleMinAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleMinVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,9 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -74,23 +71,6 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleSumBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || ValueType.isNumeric(capabilities.getType()));
}
return expression == null;
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class FloatMaxAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatMaxBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,17 +80,6 @@ public class FloatMaxAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatMaxVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class FloatMinAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatMinBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class FloatMinAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatMinVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,22 +71,6 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatSumBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class LongMaxAggregatorFactory extends SimpleLongAggregatorFactory
return new LongMaxBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class LongMaxAggregatorFactory extends SimpleLongAggregatorFactory
return new LongMaxVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class LongMinAggregatorFactory extends SimpleLongAggregatorFactory
return new LongMinBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class LongMinAggregatorFactory extends SimpleLongAggregatorFactory
return new LongMinVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -73,12 +71,6 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
return new LongSumBufferAggregator(selector);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
@ -88,16 +80,6 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
return new LongSumVectorAggregator(selector);
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
if (fieldName != null) {
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return expression == null && (capabilities == null || capabilities.getType().isNumeric());
}
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -28,11 +28,14 @@ import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.util.Collections;
@ -119,6 +122,12 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre
);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression);
}
private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory)
{
if (fieldName != null) {
@ -235,6 +244,12 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre
return expression;
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression);
}
protected abstract double nullValue();
protected abstract Aggregator buildAggregator(BaseDoubleColumnValueSelector selector);

View File

@ -28,10 +28,13 @@ import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.util.Collections;
@ -110,6 +113,12 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg
);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression);
}
@Override
public Object deserialize(Object object)
{
@ -214,6 +223,12 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg
return expression;
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression);
}
private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory)
{
if (fieldName != null) {

View File

@ -28,10 +28,13 @@ import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.util.Collections;
@ -116,6 +119,12 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega
);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression);
}
@Override
public Object deserialize(Object object)
{
@ -217,6 +226,12 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega
return expression;
}
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression);
}
private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory)
{
if (fieldName != null) {

View File

@ -26,6 +26,9 @@ import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.math.expr.vector.LongOutLongInFunctionVectorProcessor;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
@ -120,6 +123,31 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
return ExprType.LONG;
}
@Override
public boolean canVectorize(InputBindingTypes inputTypes)
{
return args.get(0).canVectorize(inputTypes);
}
@Override
public <T> ExprVectorProcessor<T> buildVectorized(VectorInputBindingTypes inputTypes)
{
ExprVectorProcessor<?> processor;
processor = new LongOutLongInFunctionVectorProcessor(
CastToTypeVectorProcessor.castToType(args.get(0).buildVectorized(inputTypes), ExprType.LONG),
inputTypes.getMaxVectorSize()
)
{
@Override
public long apply(long input)
{
return granularity.bucketStart(DateTimes.utc(input)).getMillis();
}
};
return (ExprVectorProcessor<T>) processor;
}
@Override
public boolean equals(Object o)
{

View File

@ -19,11 +19,13 @@
package org.apache.druid.segment;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.segment.column.ColumnCapabilities;
import javax.annotation.Nullable;
public interface ColumnInspector
public interface ColumnInspector extends Expr.InputBindingTypes
{
/**
* Returns capabilities of a particular column.
@ -34,4 +36,15 @@ public interface ColumnInspector
*/
@Nullable
ColumnCapabilities getColumnCapabilities(String column);
@Nullable
@Override
default ExprType getType(String name)
{
ColumnCapabilities capabilities = getColumnCapabilities(name);
if (capabilities != null) {
return ExprType.fromValueType(capabilities.getType());
}
return null;
}
}

View File

@ -161,7 +161,15 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
{
if (isVirtualColumn(dimension)) {
return virtualColumns.getVirtualColumn(dimension).capabilities(dimension).hasMultipleValues();
VirtualColumn virtualColumn = virtualColumns.getVirtualColumn(dimension);
ColumnCapabilities virtualCapabilities = null;
if (virtualColumn != null) {
virtualCapabilities = virtualColumn.capabilities(
QueryableIndexStorageAdapter.getColumnInspectorForIndex(index),
dimension
);
}
return virtualCapabilities != null ? virtualCapabilities.hasMultipleValues() : ColumnCapabilities.Capable.FALSE;
}
final ColumnHolder columnHolder = index.getColumnHolder(dimension);

View File

@ -189,7 +189,10 @@ public class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactor
public ColumnCapabilities getColumnCapabilities(String columnName)
{
if (virtualColumns.exists(columnName)) {
return virtualColumns.getColumnCapabilities(columnName);
return virtualColumns.getColumnCapabilities(
baseColumnName -> QueryableIndexStorageAdapter.getColumnCapabilities(index, baseColumnName),
columnName
);
}
return QueryableIndexStorageAdapter.getColumnCapabilities(index, columnName);

View File

@ -315,6 +315,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
return columnHolder.getCapabilities();
}
public static ColumnInspector getColumnInspectorForIndex(ColumnSelector index)
{
return new ColumnInspector()
{
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return QueryableIndexStorageAdapter.getColumnCapabilities(index, column);
}
};
}
@Override
public Metadata getMetadata()
{

View File

@ -235,17 +235,39 @@ public interface VirtualColumn extends Cacheable
}
/**
* Returns the capabilities of this virtual column, which includes a type that corresponds to the best
* performing base selector supertype (e. g. {@link BaseLongColumnValueSelector}) of the object, returned from
* {@link #makeColumnValueSelector(String, ColumnSelectorFactory)}. May vary based on columnName if this column uses
* dot notation.
* This method is deprecated in favor of {@link #capabilities(ColumnInspector, String)}, which should be used whenever
* possible and can support virtual column implementations that need to inspect other columns as inputs.
*
* This is a fallback implementation to return the capabilities of this virtual column, which includes a type that
* corresponds to the best performing base selector supertype (e. g. {@link BaseLongColumnValueSelector}) of the
* object, returned from {@link #makeColumnValueSelector(String, ColumnSelectorFactory)}. May vary based on columnName
* if this column uses dot notation.
*
* @param columnName the name this virtual column was referenced with
*
* @return capabilities, must not be null
*/
@Deprecated
ColumnCapabilities capabilities(String columnName);
/**
* Return the {@link ColumnCapabilities} which best describe the optimal selector to read from this virtual column.
*
* The {@link ColumnInspector} (most likely corresponding to an underlying {@link ColumnSelectorFactory} of a query)
* allows the virtual column to consider this information if necessary to compute its output type details.
*
* Examples of this include the {@link ExpressionVirtualColumn}, which takes input from other columns and uses the
* {@link ColumnInspector} to infer the output type of expressions based on the types of the inputs.
*
* @param inspector column inspector to provide additional information of other available columns
* @param columnName the name this virtual column was referenced with
* @return capabilities, must not be null
*/
default ColumnCapabilities capabilities(ColumnInspector inspector, String columnName)
{
return capabilities(columnName);
}
/**
* Returns a list of columns that this virtual column will access. This may include the
* names of other virtual columns. May be empty if a virtual column doesn't access any

View File

@ -371,12 +371,12 @@ public class VirtualColumns implements Cacheable
}
@Nullable
public ColumnCapabilities getColumnCapabilities(String columnName)
public ColumnCapabilities getColumnCapabilities(ColumnInspector inspector, String columnName)
{
final VirtualColumn virtualColumn = getVirtualColumn(columnName);
if (virtualColumn != null) {
return Preconditions.checkNotNull(
virtualColumn.capabilities(columnName),
virtualColumn.capabilities(inspector, columnName),
"capabilities for column[%s]",
columnName
);
@ -386,13 +386,13 @@ public class VirtualColumns implements Cacheable
}
@Nullable
public ColumnCapabilities getColumnCapabilitiesWithFallback(StorageAdapter adapter, String columnName)
public ColumnCapabilities getColumnCapabilitiesWithFallback(ColumnInspector inspector, String columnName)
{
final ColumnCapabilities virtualColumnCapabilities = getColumnCapabilities(columnName);
final ColumnCapabilities virtualColumnCapabilities = getColumnCapabilities(inspector, columnName);
if (virtualColumnCapabilities != null) {
return virtualColumnCapabilities;
} else {
return adapter.getColumnCapabilities(columnName);
return inspector.getColumnCapabilities(columnName);
}
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.ColumnInspector;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -46,7 +47,7 @@ import java.util.Optional;
* @see org.apache.druid.query.QueryToolChest#resultArraySignature which returns signatures for query results
* @see org.apache.druid.query.InlineDataSource#getRowSignature which returns signatures for inline datasources
*/
public class RowSignature
public class RowSignature implements ColumnInspector
{
private static final RowSignature EMPTY = new RowSignature(Collections.emptyList());
@ -191,6 +192,18 @@ public class RowSignature
return s.append("}").toString();
}
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return getColumnType(column).map(valueType -> {
if (valueType.isNumeric()) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType);
}
return new ColumnCapabilitiesImpl().setType(valueType);
}).orElse(null);
}
public static class Builder
{
private final List<Pair<String, ValueType>> columnTypeList;

View File

@ -40,6 +40,7 @@ import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelecto
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorObjectSelector;
import javax.annotation.Nullable;
import java.io.IOException;
@ -487,6 +488,56 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<St
return new QueryableMultiValueDimensionVectorSelector();
}
@Override
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
{
if (!hasMultipleValues()) {
class DictionaryEncodedStringSingleValueVectorObjectSelector implements VectorObjectSelector
{
private final int[] vector = new int[offset.getMaxVectorSize()];
private final String[] strings = new String[offset.getMaxVectorSize()];
private int id = ReadableVectorOffset.NULL_ID;
@Override
public Object[] getObjectVector()
{
if (id == offset.getId()) {
return strings;
}
if (offset.isContiguous()) {
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
} else {
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
}
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
strings[i] = lookupName(vector[i]);
}
id = offset.getId();
return strings;
}
@Override
public int getMaxVectorSize()
{
return offset.getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return offset.getCurrentVectorSize();
}
}
return new DictionaryEncodedStringSingleValueVectorObjectSelector();
} else {
throw new UnsupportedOperationException("Multivalue string object selector not implemented yet");
}
}
@Override
public void close() throws IOException
{

View File

@ -81,7 +81,7 @@ public interface ColumnarFloats extends Closeable
@Override
public double getDouble(int offset)
{
return ColumnarFloats.this.get(offset);
return (double) ColumnarFloats.this.get(offset);
}
@Override

View File

@ -109,6 +109,7 @@ public class ExpressionFilter implements Filter
{
final Expr.BindingAnalysis details = this.bindingDetails.get();
if (details.getRequiredBindings().isEmpty()) {
// Constant expression.
return true;

View File

@ -311,6 +311,52 @@ public class GeneratorBasicSchemas
SCHEMA_INFO_BUILDER.put("nulls-and-non-nulls", nullsSchema);
}
static {
// schema for benchmarking expressions
List<GeneratorColumnSchema> expressionsTestsSchemaColumns = ImmutableList.of(
// string dims
GeneratorColumnSchema.makeSequential("string1", ValueType.STRING, false, 1, null, 0, 10000),
GeneratorColumnSchema.makeLazyZipf("string2", ValueType.STRING, false, 1, null, 1, 100, 1.5),
GeneratorColumnSchema.makeLazyZipf("string3", ValueType.STRING, false, 1, 0.1, 1, 1_000_000, 2.0),
GeneratorColumnSchema.makeLazyDiscreteUniform("string4", ValueType.STRING, false, 1, null, 1, 10_000),
GeneratorColumnSchema.makeLazyDiscreteUniform("string5", ValueType.STRING, false, 1, 0.3, 1, 1_000_000),
// numeric dims
GeneratorColumnSchema.makeSequential("long1", ValueType.LONG, false, 1, null, 0, 10000),
GeneratorColumnSchema.makeLazyZipf("long2", ValueType.LONG, false, 1, null, 1, 101, 1.5),
GeneratorColumnSchema.makeLazyZipf("long3", ValueType.LONG, false, 1, 0.1, -1_000_000, 1_000_000, 2.0),
GeneratorColumnSchema.makeLazyDiscreteUniform("long4", ValueType.LONG, false, 1, null, -10_000, 10000),
GeneratorColumnSchema.makeLazyDiscreteUniform("long5", ValueType.LONG, false, 1, 0.3, -1_000_000, 1_000_000),
GeneratorColumnSchema.makeLazyZipf("double1", ValueType.DOUBLE, false, 1, null, 1, 101, 1.5),
GeneratorColumnSchema.makeLazyZipf("double2", ValueType.DOUBLE, false, 1, 0.1, -1_000_000, 1_000_000, 2.0),
GeneratorColumnSchema.makeContinuousUniform("double3", ValueType.DOUBLE, false, 1, null, -9000.0, 9000.0),
GeneratorColumnSchema.makeContinuousUniform("double4", ValueType.DOUBLE, false, 1, null, -1_000_000, 1_000_000),
GeneratorColumnSchema.makeLazyZipf("double5", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0),
GeneratorColumnSchema.makeLazyZipf("float1", ValueType.FLOAT, false, 1, null, 1, 101, 1.5),
GeneratorColumnSchema.makeLazyZipf("float2", ValueType.FLOAT, false, 1, 0.1, -1_000_000, 1_000_000, 2.0),
GeneratorColumnSchema.makeContinuousUniform("float3", ValueType.FLOAT, false, 1, null, -9000.0, 9000.0),
GeneratorColumnSchema.makeContinuousUniform("float4", ValueType.FLOAT, false, 1, null, -1_000_000, 1_000_000),
GeneratorColumnSchema.makeLazyZipf("float5", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0)
);
List<AggregatorFactory> aggs = new ArrayList<>();
aggs.add(new CountAggregatorFactory("rows"));
Interval interval = Intervals.of("2000-01-01/P1D");
GeneratorSchemaInfo expressionsTestsSchema = new GeneratorSchemaInfo(
expressionsTestsSchemaColumns,
aggs,
interval,
false
);
SCHEMA_INFO_BUILDER.put("expression-testbench", expressionsTestsSchema);
}
static {
// simple 'wide' schema with null valued rows, high cardinality columns, no aggs on numeric columns
// essentially 'nulls-and-non-nulls' with a ton of extra zipf columns of each type with a variety of value

View File

@ -127,7 +127,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory
public ColumnCapabilities getColumnCapabilities(String columnName)
{
if (virtualColumns.exists(columnName)) {
return virtualColumns.getColumnCapabilities(columnName);
return virtualColumns.getColumnCapabilities(adapter, columnName);
}
// Use adapter.getColumnCapabilities instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater)

View File

@ -266,7 +266,10 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
public ColumnCapabilities getColumnCapabilities(final String columnName)
{
if (virtualColumns.exists(columnName)) {
return virtualColumns.getColumnCapabilities(columnName);
return virtualColumns.getColumnCapabilities(
baseColumnName -> QueryableIndexStorageAdapter.getColumnCapabilities(index, baseColumnName),
columnName
);
}
return QueryableIndexStorageAdapter.getColumnCapabilities(index, columnName);
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.segment.virtual;
import com.google.common.base.Predicate;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
@ -41,11 +42,22 @@ import java.util.stream.Collectors;
* Basic multi-value dimension selector for an {@link org.apache.druid.math.expr.Expr} evaluating
* {@link ColumnValueSelector}.
*/
public class MultiValueExpressionDimensionSelector implements DimensionSelector
public class ExpressionMultiValueDimensionSelector implements DimensionSelector
{
private final ColumnValueSelector<ExprEval> baseSelector;
public static ExpressionMultiValueDimensionSelector fromValueSelector(
ColumnValueSelector<ExprEval> baseSelector,
@Nullable ExtractionFn extractionFn
)
{
if (extractionFn != null) {
return new ExtractionMultiValueDimensionSelector(baseSelector, extractionFn);
}
return new ExpressionMultiValueDimensionSelector(baseSelector);
}
public MultiValueExpressionDimensionSelector(ColumnValueSelector<ExprEval> baseSelector)
protected final ColumnValueSelector<ExprEval> baseSelector;
public ExpressionMultiValueDimensionSelector(ColumnValueSelector<ExprEval> baseSelector)
{
this.baseSelector = baseSelector;
}
@ -196,4 +208,50 @@ public class MultiValueExpressionDimensionSelector implements DimensionSelector
{
return Object.class;
}
/**
* expressions + extractions
*/
static class ExtractionMultiValueDimensionSelector extends ExpressionMultiValueDimensionSelector
{
private final ExtractionFn extractionFn;
private ExtractionMultiValueDimensionSelector(ColumnValueSelector<ExprEval> baseSelector, ExtractionFn extractionFn)
{
super(baseSelector);
this.extractionFn = extractionFn;
}
@Override
String getValue(ExprEval evaluated)
{
assert !evaluated.isArray();
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(evaluated.asString()));
}
@Override
List<String> getArray(ExprEval evaluated)
{
assert evaluated.isArray();
return Arrays.stream(evaluated.asStringArray())
.map(x -> extractionFn.apply(NullHandling.emptyToNullIfNeeded(x)))
.collect(Collectors.toList());
}
@Override
String getArrayValue(ExprEval evaluated, int i)
{
assert evaluated.isArray();
String[] stringArray = evaluated.asStringArray();
assert i < stringArray.length;
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(stringArray[i]));
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
inspector.visit("extractionFn", extractionFn);
}
}
}

View File

@ -0,0 +1,167 @@
/*
* 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.virtual;
import com.google.common.collect.Iterables;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.List;
import java.util.Set;
public class ExpressionPlan
{
public enum Trait
{
/**
* expression has no inputs and can be optimized into a constant selector
*/
CONSTANT,
/**
* expression has a single, single valued input, and is dictionary encoded if the value is a string
*/
SINGLE_INPUT_SCALAR,
/**
* expression has a single input, which may produce single or multi-valued output, but if so, it must be implicitly
* mappable (i.e. the expression is not treating its input as an array and not wanting to output an array)
*/
SINGLE_INPUT_MAPPABLE,
/**
* expression must be implicitly mapped across the multiple values per row of known multi-value inputs
*/
NEEDS_APPLIED,
/**
* expression has inputs whose type was unresolveable, or was incomplete, such as unknown multi-valuedness
*/
UNKNOWN_INPUTS,
/**
* expression explicitly using multi-valued inputs as array inputs
*/
NON_SCALAR_INPUTS,
/**
* expression produces explict multi-valued output, or implicit multi-valued output via mapping
*/
NON_SCALAR_OUTPUT,
/**
* expression is vectorizable
*/
VECTORIZABLE
}
private final Expr expression;
private final Expr.BindingAnalysis analysis;
private final EnumSet<Trait> traits;
@Nullable
private final ExprType outputType;
@Nullable
private final ValueType singleInputType;
private final Set<String> unknownInputs;
private final List<String> unappliedInputs;
ExpressionPlan(
Expr expression,
Expr.BindingAnalysis analysis,
EnumSet<Trait> traits,
@Nullable ExprType outputType,
@Nullable ValueType singleInputType,
Set<String> unknownInputs,
List<String> unappliedInputs
)
{
this.expression = expression;
this.analysis = analysis;
this.traits = traits;
this.outputType = outputType;
this.singleInputType = singleInputType;
this.unknownInputs = unknownInputs;
this.unappliedInputs = unappliedInputs;
}
public Expr getExpression()
{
return expression;
}
public Expr getAppliedExpression()
{
if (is(Trait.NEEDS_APPLIED)) {
return Parser.applyUnappliedBindings(expression, analysis, unappliedInputs);
}
return expression;
}
public Expr.BindingAnalysis getAnalysis()
{
return analysis;
}
public boolean is(Trait... flags)
{
return is(traits, flags);
}
public boolean any(Trait... flags)
{
return any(traits, flags);
}
@Nullable
public ExprType getOutputType()
{
return outputType;
}
@Nullable
public ValueType getSingleInputType()
{
return singleInputType;
}
public String getSingleInputName()
{
return Iterables.getOnlyElement(analysis.getRequiredBindings());
}
public Set<String> getUnknownInputs()
{
return unknownInputs;
}
static boolean is(EnumSet<Trait> traits, Trait... args)
{
return Arrays.stream(args).allMatch(traits::contains);
}
static boolean any(EnumSet<Trait> traits, Trait... args)
{
return Arrays.stream(args).anyMatch(traits::contains);
}
static boolean none(EnumSet<Trait> traits, Trait... args)
{
return Arrays.stream(args).noneMatch(traits::contains);
}
}

View File

@ -0,0 +1,180 @@
/*
* 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.virtual;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
public class ExpressionPlanner
{
private ExpressionPlanner()
{
// No instantiation.
}
/**
* Druid tries to be chill to expressions to make up for not having a well defined table schema across segments. This
* method performs some analysis to determine what sort of selectors can be constructed on top of an expression,
* whether or not the expression will need implicitly mapped across multi-valued inputs, if the expression produces
* multi-valued outputs, is vectorizable, and everything else interesting when making a selector.
*
* Results are stored in a {@link ExpressionPlan}, which can be examined to do whatever is necessary to make things
* function properly.
*/
public static ExpressionPlan plan(ColumnInspector inspector, Expr expression)
{
final Expr.BindingAnalysis analysis = expression.analyzeInputs();
Parser.validateExpr(expression, analysis);
EnumSet<ExpressionPlan.Trait> traits = EnumSet.noneOf(ExpressionPlan.Trait.class);
Set<String> maybeMultiValued = new HashSet<>();
List<String> needsApplied = ImmutableList.of();
ValueType singleInputType = null;
ExprType outputType = null;
final Set<String> columns = analysis.getRequiredBindings();
// check and set traits which allow optimized selectors to be created
if (columns.isEmpty()) {
traits.add(ExpressionPlan.Trait.CONSTANT);
} else if (columns.size() == 1) {
final String column = Iterables.getOnlyElement(columns);
final ColumnCapabilities capabilities = inspector.getColumnCapabilities(column);
// These flags allow for selectors that wrap a single underlying column to be optimized, through caching results
// and via allowing deferred execution in the case of building dimension selectors.
// SINGLE_INPUT_SCALAR
// is set if an input is single valued, and the output is definitely single valued, with an additional requirement
// for strings that the column is dictionary encoded.
// SINGLE_INPUT_MAPPABLE
// is set when a single input string column, which can be multi-valued, but if so, it must be implicitly mappable
// (i.e. the expression is not treating its input as an array and not wanting to output an array)
if (capabilities != null) {
boolean isSingleInputMappable = false;
boolean isSingleInputScalar = capabilities.hasMultipleValues().isFalse() &&
!analysis.hasInputArrays() &&
!analysis.isOutputArray();
if (capabilities.getType() == ValueType.STRING) {
isSingleInputScalar &= capabilities.isDictionaryEncoded().isTrue();
isSingleInputMappable = capabilities.isDictionaryEncoded().isTrue() &&
!capabilities.hasMultipleValues().isUnknown() &&
!analysis.hasInputArrays() &&
!analysis.isOutputArray();
}
// if satisfied, set single input output type and flags
if (isSingleInputScalar || isSingleInputMappable) {
singleInputType = capabilities.getType();
if (isSingleInputScalar) {
traits.add(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR);
}
if (isSingleInputMappable) {
traits.add(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE);
}
}
}
}
// if we didn't eliminate this expression as a single input scalar or mappable expression, it might need
// automatic transformation to map across multi-valued inputs (or row by row detection in the worst case)
if (ExpressionPlan.none(traits, ExpressionPlan.Trait.SINGLE_INPUT_SCALAR)) {
final Set<String> definitelyMultiValued = new HashSet<>();
for (String column : analysis.getRequiredBindings()) {
final ColumnCapabilities capabilities = inspector.getColumnCapabilities(column);
if (capabilities != null) {
if (capabilities.hasMultipleValues().isTrue()) {
definitelyMultiValued.add(column);
} else if (capabilities.getType().equals(ValueType.STRING) &&
capabilities.hasMultipleValues().isMaybeTrue() &&
!analysis.getArrayBindings().contains(column)
) {
maybeMultiValued.add(column);
}
} else {
maybeMultiValued.add(column);
}
}
// find any inputs which will need implicitly mapped across multi-valued rows
needsApplied =
columns.stream()
.filter(c -> definitelyMultiValued.contains(c) && !analysis.getArrayBindings().contains(c))
.collect(Collectors.toList());
// if any multi-value inputs, set flag for non-scalar inputs
if (analysis.hasInputArrays()) {
traits.add(ExpressionPlan.Trait.NON_SCALAR_INPUTS);
}
if (!maybeMultiValued.isEmpty()) {
traits.add(ExpressionPlan.Trait.UNKNOWN_INPUTS);
}
// if expression needs transformed, lets do it
if (!needsApplied.isEmpty()) {
traits.add(ExpressionPlan.Trait.NEEDS_APPLIED);
}
}
// only set output type
if (ExpressionPlan.none(traits, ExpressionPlan.Trait.UNKNOWN_INPUTS, ExpressionPlan.Trait.NEEDS_APPLIED)) {
outputType = expression.getOutputType(inspector);
}
// if analysis, inferred output type, or implicit mapping is in play, output will be multi-valued
if (analysis.isOutputArray() || ExprType.isArray(outputType) || ExpressionPlan.is(traits, ExpressionPlan.Trait.NEEDS_APPLIED)) {
traits.add(ExpressionPlan.Trait.NON_SCALAR_OUTPUT);
}
// vectorized expressions do not currently support unknown inputs, multi-valued inputs or outputs, implicit mapping
boolean supportsVector = ExpressionPlan.none(
traits,
ExpressionPlan.Trait.UNKNOWN_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
);
if (supportsVector && expression.canVectorize(inspector)) {
traits.add(ExpressionPlan.Trait.VECTORIZABLE);
}
return new ExpressionPlan(
expression,
analysis,
traits,
outputType,
singleInputType,
maybeMultiValued,
needsApplied
);
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.collect.Iterables;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
@ -34,7 +33,6 @@ import org.apache.druid.query.expression.ExprUtils;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.BaseSingleValueDimensionSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.ConstantExprEvalSelector;
@ -48,10 +46,8 @@ import org.apache.druid.segment.data.IndexedInts;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
public class ExpressionSelectors
@ -136,70 +132,45 @@ public class ExpressionSelectors
Expr expression
)
{
final Expr.BindingAnalysis bindingAnalysis = expression.analyzeInputs();
Parser.validateExpr(expression, bindingAnalysis);
final List<String> columns = bindingAnalysis.getRequiredBindingsList();
return makeExprEvalSelector(columnSelectorFactory, ExpressionPlanner.plan(columnSelectorFactory, expression));
}
if (columns.size() == 1) {
final String column = Iterables.getOnlyElement(columns);
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
if (capabilities != null && capabilities.getType() == ValueType.LONG) {
// Optimization for expressions that hit one long column and nothing else.
public static ColumnValueSelector<ExprEval> makeExprEvalSelector(
ColumnSelectorFactory columnSelectorFactory,
ExpressionPlan plan
)
{
if (plan.is(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR)) {
final String column = plan.getSingleInputName();
final ValueType inputType = plan.getSingleInputType();
if (inputType == ValueType.LONG) {
return new SingleLongInputCachingExpressionColumnValueSelector(
columnSelectorFactory.makeColumnValueSelector(column),
expression,
plan.getExpression(),
!ColumnHolder.TIME_COLUMN_NAME.equals(column) // __time doesn't need an LRU cache since it is sorted.
);
} else if (capabilities != null
&& capabilities.getType() == ValueType.STRING
&& capabilities.isDictionaryEncoded().isTrue()
&& capabilities.hasMultipleValues().isFalse()
&& bindingAnalysis.getArrayBindings().isEmpty()) {
// Optimization for expressions that hit one scalar string column and nothing else.
} else if (inputType == ValueType.STRING) {
return new SingleStringInputCachingExpressionColumnValueSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ValueType.STRING)),
expression
plan.getExpression()
);
}
}
final Expr.ObjectBinding bindings = createBindings(plan.getAnalysis(), columnSelectorFactory);
final Pair<Set<String>, Set<String>> arrayUsage =
examineColumnSelectorFactoryArrays(columnSelectorFactory, bindingAnalysis, columns);
final Set<String> actualArrays = arrayUsage.lhs;
final Set<String> unknownIfArrays = arrayUsage.rhs;
final List<String> needsApplied =
columns.stream()
.filter(c -> actualArrays.contains(c) && !bindingAnalysis.getArrayBindings().contains(c))
.collect(Collectors.toList());
final Expr finalExpr;
if (needsApplied.size() > 0) {
finalExpr = Parser.applyUnappliedBindings(expression, bindingAnalysis, needsApplied);
} else {
finalExpr = expression;
}
final Expr.ObjectBinding bindings = createBindings(bindingAnalysis, columnSelectorFactory);
// Optimization for constant expressions
if (bindings.equals(ExprUtils.nilBindings())) {
// Optimization for constant expressions.
return new ConstantExprEvalSelector(expression.eval(bindings));
return new ConstantExprEvalSelector(plan.getExpression().eval(bindings));
}
// if any unknown column input types, fall back to an expression selector that examines input bindings on a
// per row basis
if (unknownIfArrays.size() > 0) {
return new RowBasedExpressionColumnValueSelector(
finalExpr,
bindingAnalysis,
bindings,
unknownIfArrays
);
if (plan.is(ExpressionPlan.Trait.UNKNOWN_INPUTS)) {
return new RowBasedExpressionColumnValueSelector(plan, bindings);
}
// generic expression value selector for fully known input types
return new ExpressionColumnValueSelector(finalExpr, bindings);
return new ExpressionColumnValueSelector(plan.getAppliedExpression(), bindings);
}
/**
@ -212,39 +183,19 @@ public class ExpressionSelectors
@Nullable final ExtractionFn extractionFn
)
{
final Expr.BindingAnalysis bindingAnalysis = expression.analyzeInputs();
Parser.validateExpr(expression, bindingAnalysis);
final List<String> columns = bindingAnalysis.getRequiredBindingsList();
final ExpressionPlan plan = ExpressionPlanner.plan(columnSelectorFactory, expression);
if (columns.size() == 1) {
final String column = Iterables.getOnlyElement(columns);
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
// Optimization for dimension selectors that wrap a single underlying string column.
// The string column can be multi-valued, but if so, it must be implicitly mappable (i.e. the expression is
// not treating it as an array and not wanting to output an array
if (capabilities != null
&& capabilities.getType() == ValueType.STRING
&& capabilities.isDictionaryEncoded().isTrue()
&& canMapOverDictionary(bindingAnalysis, capabilities.hasMultipleValues())
) {
if (plan.is(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE)) {
final String column = plan.getSingleInputName();
if (plan.getSingleInputType() == ValueType.STRING) {
return new SingleStringInputDimensionSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ValueType.STRING)),
columnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of(column)),
expression
);
}
}
final Pair<Set<String>, Set<String>> arrayUsage =
examineColumnSelectorFactoryArrays(columnSelectorFactory, bindingAnalysis, columns);
final Set<String> actualArrays = arrayUsage.lhs;
final Set<String> unknownIfArrays = arrayUsage.rhs;
final ColumnValueSelector<ExprEval> baseSelector = makeExprEvalSelector(columnSelectorFactory, expression);
final boolean multiVal = actualArrays.size() > 0 ||
bindingAnalysis.getArrayBindings().size() > 0 ||
unknownIfArrays.size() > 0;
if (baseSelector instanceof ConstantExprEvalSelector) {
// Optimization for dimension selectors on constants.
@ -252,91 +203,20 @@ public class ExpressionSelectors
} else if (baseSelector instanceof NilColumnValueSelector) {
// Optimization for null dimension selector.
return DimensionSelector.constant(null);
} else if (extractionFn == null) {
if (multiVal) {
return new MultiValueExpressionDimensionSelector(baseSelector);
} else {
class DefaultExpressionDimensionSelector extends BaseSingleValueDimensionSelector
{
@Override
protected String getValue()
{
return NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString());
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
}
}
return new DefaultExpressionDimensionSelector();
}
} else {
if (multiVal) {
class ExtractionMultiValueDimensionSelector extends MultiValueExpressionDimensionSelector
{
private ExtractionMultiValueDimensionSelector()
{
super(baseSelector);
}
@Override
String getValue(ExprEval evaluated)
{
assert !evaluated.isArray();
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(evaluated.asString()));
}
@Override
List<String> getArray(ExprEval evaluated)
{
assert evaluated.isArray();
return Arrays.stream(evaluated.asStringArray())
.map(x -> extractionFn.apply(NullHandling.emptyToNullIfNeeded(x)))
.collect(Collectors.toList());
}
@Override
String getArrayValue(ExprEval evaluated, int i)
{
assert evaluated.isArray();
String[] stringArray = evaluated.asStringArray();
assert i < stringArray.length;
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(stringArray[i]));
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
inspector.visit("extractionFn", extractionFn);
}
}
return new ExtractionMultiValueDimensionSelector();
if (plan.any(
ExpressionPlan.Trait.NON_SCALAR_OUTPUT,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.UNKNOWN_INPUTS
)) {
return ExpressionMultiValueDimensionSelector.fromValueSelector(baseSelector, extractionFn);
} else {
class ExtractionExpressionDimensionSelector extends BaseSingleValueDimensionSelector
{
@Override
protected String getValue()
{
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString()));
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
inspector.visit("extractionFn", extractionFn);
}
}
return new ExtractionExpressionDimensionSelector();
return ExpressionSingleValueDimensionSelector.fromValueSelector(baseSelector, extractionFn);
}
}
}
/**
* Returns whether an expression can be applied to unique values of a particular column (like those in a dictionary)
* rather than being applied to each row individually.
@ -369,8 +249,7 @@ public class ExpressionSelectors
final Map<String, Supplier<Object>> suppliers = new HashMap<>();
final List<String> columns = bindingAnalysis.getRequiredBindingsList();
for (String columnName : columns) {
final ColumnCapabilities columnCapabilities = columnSelectorFactory
.getColumnCapabilities(columnName);
final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(columnName);
final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null;
final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues().isTrue();
final Supplier<Object> supplier;
@ -594,36 +473,4 @@ public class ExpressionSelectors
return eval.value();
}
}
/**
* Returns pair of columns which are definitely multi-valued, or 'actual' arrays, and those which we are unable to
* discern from the {@link ColumnSelectorFactory#getColumnCapabilities(String)}, or 'unknown' arrays.
*/
private static Pair<Set<String>, Set<String>> examineColumnSelectorFactoryArrays(
ColumnSelectorFactory columnSelectorFactory,
Expr.BindingAnalysis bindingAnalysis,
List<String> columns
)
{
final Set<String> actualArrays = new HashSet<>();
final Set<String> unknownIfArrays = new HashSet<>();
for (String column : columns) {
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
if (capabilities != null) {
if (capabilities.hasMultipleValues().isTrue()) {
actualArrays.add(column);
} else if (
capabilities.getType().equals(ValueType.STRING) &&
capabilities.hasMultipleValues().isMaybeTrue() &&
!bindingAnalysis.getArrayBindings().contains(column)
) {
unknownIfArrays.add(column);
}
} else {
unknownIfArrays.add(column);
}
}
return new Pair<>(actualArrays, unknownIfArrays);
}
}

View File

@ -0,0 +1,90 @@
/*
* 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.virtual;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseSingleValueDimensionSelector;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
class ExpressionSingleValueDimensionSelector extends BaseSingleValueDimensionSelector
{
public static ExpressionSingleValueDimensionSelector fromValueSelector(
ColumnValueSelector<ExprEval> baseSelector,
@Nullable ExtractionFn extractionFn
)
{
if (extractionFn != null) {
return new ExtractionExpressionDimensionSelector(baseSelector, extractionFn);
}
return new ExpressionSingleValueDimensionSelector(baseSelector);
}
protected final ColumnValueSelector<ExprEval> baseSelector;
ExpressionSingleValueDimensionSelector(ColumnValueSelector<ExprEval> baseSelector)
{
this.baseSelector = baseSelector;
}
@Override
protected String getValue()
{
return NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString());
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
}
/**
* expressions + extractions
*/
static class ExtractionExpressionDimensionSelector extends ExpressionSingleValueDimensionSelector
{
private final ExtractionFn extractionFn;
ExtractionExpressionDimensionSelector(ColumnValueSelector<ExprEval> baseSelector, ExtractionFn extractionFn)
{
super(baseSelector);
this.extractionFn = extractionFn;
}
@Override
protected String getValue()
{
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString()));
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
inspector.visit("extractionFn", extractionFn);
}
}
}

View File

@ -0,0 +1,107 @@
/*
* 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.virtual;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.segment.vector.NilVectorSelector;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorSizeInspector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.util.HashMap;
import java.util.Map;
class ExpressionVectorInputBinding implements Expr.VectorInputBinding
{
private final Map<String, VectorValueSelector> numeric;
private final Map<String, VectorObjectSelector> objects;
private final Map<String, ExprType> types;
private final NilVectorSelector nilSelector;
private final VectorSizeInspector sizeInspector;
public ExpressionVectorInputBinding(VectorSizeInspector sizeInspector)
{
this.numeric = new HashMap<>();
this.objects = new HashMap<>();
this.types = new HashMap<>();
this.sizeInspector = sizeInspector;
this.nilSelector = NilVectorSelector.create(sizeInspector);
}
public ExpressionVectorInputBinding addNumeric(String name, ExprType type, VectorValueSelector selector)
{
numeric.put(name, selector);
types.put(name, type);
return this;
}
public ExpressionVectorInputBinding addObjectSelector(String name, ExprType type, VectorObjectSelector selector)
{
objects.put(name, selector);
types.put(name, type);
return this;
}
@Override
public <T> T[] getObjectVector(String name)
{
return (T[]) objects.getOrDefault(name, nilSelector).getObjectVector();
}
@Override
public ExprType getType(String name)
{
return types.get(name);
}
@Override
public long[] getLongVector(String name)
{
return numeric.getOrDefault(name, nilSelector).getLongVector();
}
@Override
public double[] getDoubleVector(String name)
{
return numeric.getOrDefault(name, nilSelector).getDoubleVector();
}
@Nullable
@Override
public boolean[] getNullVector(String name)
{
return numeric.getOrDefault(name, nilSelector).getNullVector();
}
@Override
public int getMaxVectorSize()
{
return sizeInspector.getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return sizeInspector.getCurrentVectorSize();
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.virtual;
import com.google.common.base.Preconditions;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.segment.vector.VectorObjectSelector;
public class ExpressionVectorObjectSelector implements VectorObjectSelector
{
final Expr.VectorInputBinding bindings;
final ExprVectorProcessor<?> processor;
public ExpressionVectorObjectSelector(ExprVectorProcessor<?> processor, Expr.VectorInputBinding bindings)
{
this.processor = Preconditions.checkNotNull(processor, "processor");
this.bindings = Preconditions.checkNotNull(bindings, "bindings");
}
@Override
public Object[] getObjectVector()
{
return processor.evalVector(bindings).getObjectVector();
}
@Override
public int getMaxVectorSize()
{
return bindings.getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return bindings.getCurrentVectorSize();
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.virtual;
import com.google.common.base.Preconditions;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import java.util.List;
public class ExpressionVectorSelectors
{
private ExpressionVectorSelectors()
{
// No instantiation.
}
public static VectorValueSelector makeVectorValueSelector(
VectorColumnSelectorFactory factory,
Expr expression
)
{
final ExpressionPlan plan = ExpressionPlanner.plan(factory, expression);
Preconditions.checkArgument(plan.is(ExpressionPlan.Trait.VECTORIZABLE));
final Expr.VectorInputBinding bindings = createVectorBindings(plan.getAnalysis(), factory);
final ExprVectorProcessor<?> processor = plan.getExpression().buildVectorized(bindings);
return new ExpressionVectorValueSelector(processor, bindings);
}
public static VectorObjectSelector makeVectorObjectSelector(
VectorColumnSelectorFactory factory,
Expr expression
)
{
final ExpressionPlan plan = ExpressionPlanner.plan(factory, expression);
Preconditions.checkArgument(plan.is(ExpressionPlan.Trait.VECTORIZABLE));
final Expr.VectorInputBinding bindings = createVectorBindings(plan.getAnalysis(), factory);
final ExprVectorProcessor<?> processor = plan.getExpression().buildVectorized(bindings);
return new ExpressionVectorObjectSelector(processor, bindings);
}
private static Expr.VectorInputBinding createVectorBindings(
Expr.BindingAnalysis bindingAnalysis,
VectorColumnSelectorFactory vectorColumnSelectorFactory
)
{
ExpressionVectorInputBinding binding = new ExpressionVectorInputBinding(vectorColumnSelectorFactory.getVectorSizeInspector());
final List<String> columns = bindingAnalysis.getRequiredBindingsList();
for (String columnName : columns) {
final ColumnCapabilities columnCapabilities = vectorColumnSelectorFactory.getColumnCapabilities(columnName);
final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null;
// null capabilities should be backed by a nil vector selector since it means the column effectively doesnt exist
if (nativeType != null) {
switch (nativeType) {
case FLOAT:
case DOUBLE:
binding.addNumeric(columnName, ExprType.DOUBLE, vectorColumnSelectorFactory.makeValueSelector(columnName));
break;
case LONG:
binding.addNumeric(columnName, ExprType.LONG, vectorColumnSelectorFactory.makeValueSelector(columnName));
break;
default:
binding.addObjectSelector(
columnName,
ExprType.STRING,
vectorColumnSelectorFactory.makeObjectSelector(columnName)
);
}
}
}
return binding;
}
}

View File

@ -0,0 +1,82 @@
/*
* 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.virtual;
import com.google.common.base.Preconditions;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
public class ExpressionVectorValueSelector implements VectorValueSelector
{
final Expr.VectorInputBinding bindings;
final ExprVectorProcessor<?> processor;
final float[] floats;
public ExpressionVectorValueSelector(ExprVectorProcessor<?> processor, Expr.VectorInputBinding bindings)
{
this.processor = Preconditions.checkNotNull(processor, "processor");
this.bindings = Preconditions.checkNotNull(bindings, "bindings");
this.floats = new float[bindings.getMaxVectorSize()];
}
@Override
public long[] getLongVector()
{
return processor.evalVector(bindings).getLongVector();
}
@Override
public float[] getFloatVector()
{
final double[] doubles = processor.evalVector(bindings).getDoubleVector();
for (int i = 0; i < bindings.getCurrentVectorSize(); i++) {
floats[i] = (float) doubles[i];
}
return floats;
}
@Override
public double[] getDoubleVector()
{
return processor.evalVector(bindings).getDoubleVector();
}
@Nullable
@Override
public boolean[] getNullVector()
{
return processor.evalVector(bindings).getNullVector();
}
@Override
public int getMaxVectorSize()
{
return bindings.getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return bindings.getCurrentVectorSize();
}
}

View File

@ -27,11 +27,14 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
@ -39,6 +42,9 @@ import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.util.List;
@ -46,8 +52,11 @@ import java.util.Objects;
public class ExpressionVirtualColumn implements VirtualColumn
{
private static final Logger log = new Logger(ExpressionVirtualColumn.class);
private final String name;
private final String expression;
@Nullable
private final ValueType outputType;
private final Supplier<Expr> parsedExpression;
@ -61,7 +70,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
{
this.name = Preconditions.checkNotNull(name, "name");
this.expression = Preconditions.checkNotNull(expression, "expression");
this.outputType = outputType != null ? outputType : ValueType.FLOAT;
this.outputType = outputType;
this.parsedExpression = Suppliers.memoize(() -> Parser.parse(expression, macroTable));
}
@ -78,7 +87,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
// Unfortunately this string representation can't be reparsed into the same expression, might be useful
// if the expression system supported that
this.expression = parsedExpression.toString();
this.outputType = outputType != null ? outputType : ValueType.FLOAT;
this.outputType = outputType;
this.parsedExpression = Suppliers.ofInstance(parsedExpression);
}
@ -95,6 +104,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
return expression;
}
@Nullable
@JsonProperty
public ValueType getOutputType()
{
@ -129,13 +139,81 @@ public class ExpressionVirtualColumn implements VirtualColumn
return ExpressionSelectors.makeColumnValueSelector(factory, parsedExpression.get());
}
@Override
public boolean canVectorize(ColumnInspector inspector)
{
final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get());
return plan.is(ExpressionPlan.Trait.VECTORIZABLE);
}
@Override
public VectorValueSelector makeVectorValueSelector(String columnName, VectorColumnSelectorFactory factory)
{
return ExpressionVectorSelectors.makeVectorValueSelector(factory, parsedExpression.get());
}
@Override
public VectorObjectSelector makeVectorObjectSelector(String columnName, VectorColumnSelectorFactory factory)
{
return ExpressionVectorSelectors.makeVectorObjectSelector(factory, parsedExpression.get());
}
@Override
public ColumnCapabilities capabilities(String columnName)
{
// Note: Ideally we would fill out additional information instead of leaving capabilities as 'unknown', e.g. examine
// if the expression in question could potentially return multiple values and anything else. However, we don't
// currently have a good way of determining this, so fill this out more once we do
return new ColumnCapabilitiesImpl().setType(outputType);
// If possible, this should only be used as a fallback method for when capabilities are truly 'unknown', because we
// are unable to compute the output type of the expression, either due to incomplete type information of the
// inputs or because of unimplemented methods on expression implementations themselves, or, because a
// ColumnInspector is not available
return new ColumnCapabilitiesImpl().setType(outputType == null ? ValueType.FLOAT : outputType);
}
@Override
public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName)
{
final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get());
if (plan.getOutputType() != null) {
if (outputType != null && ExprType.fromValueType(outputType) != plan.getOutputType()) {
log.warn(
"Projected output type %s of expression %s does not match provided type %s",
plan.getOutputType(),
expression,
outputType
);
}
final ExprType inferredOutputType = plan.getOutputType();
final ValueType valueType = ExprType.toValueType(inferredOutputType);
if (valueType.isNumeric()) {
// if float was explicitly specified preserve it, because it will currently never be the computed output type
if (ValueType.FLOAT == outputType) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT);
}
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType);
}
// null constants can sometimes trip up the type inference to report STRING, so check if explicitly supplied
// output type is numeric and stick with that if so
if (outputType != null && outputType.isNumeric()) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(outputType);
}
// we don't have to check for unknown input here because output type is unable to be inferred if we don't know
// the complete set of input types
if (plan.any(ExpressionPlan.Trait.NON_SCALAR_OUTPUT, ExpressionPlan.Trait.NEEDS_APPLIED)) {
// always a multi-value string since wider engine does not yet support array types
return new ColumnCapabilitiesImpl().setType(ValueType.STRING).setHasMultipleValues(true);
}
// if we got here, lets call it single value string output
return new ColumnCapabilitiesImpl().setType(ValueType.STRING)
.setHasMultipleValues(false)
.setDictionaryEncoded(false);
}
// fallback to
return capabilities(columnName);
}
@Override
@ -153,11 +231,14 @@ public class ExpressionVirtualColumn implements VirtualColumn
@Override
public byte[] getCacheKey()
{
return new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_EXPRESSION)
CacheKeyBuilder builder = new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_EXPRESSION)
.appendString(name)
.appendString(expression)
.appendString(outputType.toString())
.build();
.appendString(expression);
if (outputType != null) {
builder.appendString(outputType.toString());
}
return builder.build();
}
@Override

View File

@ -45,17 +45,16 @@ public class RowBasedExpressionColumnValueSelector extends ExpressionColumnValue
private final Int2ObjectMap<Expr> transformedCache;
public RowBasedExpressionColumnValueSelector(
Expr expression,
Expr.BindingAnalysis baseBindingAnalysis,
Expr.ObjectBinding bindings,
Set<String> unknownColumnsSet
ExpressionPlan plan,
Expr.ObjectBinding bindings
)
{
super(expression, bindings);
this.unknownColumns = unknownColumnsSet.stream()
.filter(x -> !baseBindingAnalysis.getArrayBindings().contains(x))
.collect(Collectors.toList());
this.baseBindingAnalysis = baseBindingAnalysis;
super(plan.getAppliedExpression(), bindings);
this.unknownColumns = plan.getUnknownInputs()
.stream()
.filter(x -> !plan.getAnalysis().getArrayBindings().contains(x))
.collect(Collectors.toList());
this.baseBindingAnalysis = plan.getAnalysis();
this.ignoredColumns = new HashSet<>();
this.transformedCache = new Int2ObjectArrayMap<>(unknownColumns.size());
}

View File

@ -68,7 +68,7 @@ public class VirtualizedColumnSelectorFactory implements ColumnSelectorFactory
public ColumnCapabilities getColumnCapabilities(String columnName)
{
if (virtualColumns.exists(columnName)) {
return virtualColumns.getColumnCapabilities(columnName);
return virtualColumns.getColumnCapabilities(baseFactory, columnName);
} else {
return baseFactory.getColumnCapabilities(columnName);
}

View File

@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.js.JavaScriptConfig;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.BySegmentResultValue;
import org.apache.druid.query.BySegmentResultValueClass;
import org.apache.druid.query.ChainedExecutionQueryRunner;
@ -3211,9 +3212,6 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
@Test
public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderByUsingMathExpressions()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
final int limit = 14;
GroupByQuery.Builder builder = makeQueryBuilder()
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
@ -3364,9 +3362,6 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
@Test
public void testGroupByOrderLimit()
{
// Cannot vectorize due to expression-based aggregator.
cannotVectorize();
GroupByQuery.Builder builder = makeQueryBuilder()
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
.setInterval("2011-04-02/2011-04-04")
@ -4807,9 +4802,6 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
@Test
public void testDifferentGroupingSubquery()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
GroupByQuery subquery = makeQueryBuilder()
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
@ -10652,6 +10644,44 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy");
}
@Test
public void testGroupByOnVirtualColumn()
{
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
expectedException.expect(UnsupportedOperationException.class);
}
GroupByQuery query = makeQueryBuilder()
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
.setVirtualColumns(
new ExpressionVirtualColumn(
"v",
"qualityDouble * qualityLong",
ValueType.LONG,
ExprMacroTable.nil()
)
)
.setDimensions(
new DefaultDimensionSpec("v", "v", ValueType.LONG)
)
.setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT)
.setGranularity(QueryRunnerTestHelper.ALL_GRAN)
.setLimit(5)
.build();
List<ResultRow> expectedResults = Arrays.asList(
makeRow(query, "2011-04-01", "v", 10000000L, "rows", 2L),
makeRow(query, "2011-04-01", "v", 12100000L, "rows", 2L),
makeRow(query, "2011-04-01", "v", 14400000L, "rows", 2L),
makeRow(query, "2011-04-01", "v", 16900000L, "rows", 2L),
makeRow(query, "2011-04-01", "v", 19600000L, "rows", 6L)
);
Iterable<ResultRow> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
TestHelper.assertExpectedObjects(expectedResults, results, "groupBy");
}
private static ResultRow makeRow(final GroupByQuery query, final String timestamp, final Object... vals)
{
return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals);

View File

@ -670,9 +670,6 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest
@Test
public void testTimeseriesWithVirtualColumn()
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.DATA_SOURCE)
.granularity(QueryRunnerTestHelper.DAY_GRAN)

View File

@ -86,7 +86,7 @@ public class RowBasedStorageAdapterTest
}
);
// Processors used by the "allProcessors" tasks.
// VectorProcessors used by the "allProcessors" tasks.
private static final LinkedHashMap<String, Function<Cursor, Supplier<Object>>> PROCESSORS = new LinkedHashMap<>();
@BeforeClass

View File

@ -411,6 +411,74 @@ public class DataGeneratorTest
Assert.assertEquals(0, dist.getNumericalVariance(), 0);
}
@Test
public void testLazyZipf()
{
List<GeneratorColumnSchema> schemas = new ArrayList<>();
RowValueTracker tracker = new RowValueTracker();
schemas.add(
GeneratorColumnSchema.makeLazyZipf(
"dimA",
ValueType.STRING,
false,
1,
null,
0,
1220000,
1.0
)
);
schemas.add(
GeneratorColumnSchema.makeLazyZipf(
"dimB",
ValueType.FLOAT,
false,
1,
null,
99990,
99999,
1.0
)
);
schemas.add(
GeneratorColumnSchema.makeLazyZipf(
"dimC",
ValueType.DOUBLE,
false,
1,
null,
0,
100000,
1.5
)
);
schemas.add(
GeneratorColumnSchema.makeLazyZipf(
"dimD",
ValueType.LONG,
false,
1,
null,
0,
100000,
1.5
)
);
DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0);
for (int i = 0; i < 100000; i++) {
InputRow row = dataGenerator.nextRow();
System.out.println("Z-ROW: " + row);
tracker.addRow(row);
}
tracker.printStuff();
}
private static class RowValueTracker
{
private Map<String, Map<Object, Integer>> dimensionMap;

View File

@ -0,0 +1,240 @@
/*
* 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.virtual;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.segment.vector.VectorCursor;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
@RunWith(Parameterized.class)
public class ExpressionVectorSelectorsTest
{
private static List<String> EXPRESSIONS = ImmutableList.of(
"long1 * long2",
"double1 * double3",
"float1 + float3",
"(long1 - long4) / double3",
"long5 * float3 * long1 * long4 * double1",
"long5 * double3 * long1 * long4 * double1",
"max(double3, double5)",
"min(double4, double1)",
"cos(float3)",
"sin(long4)",
"parse_long(string1)",
"parse_long(string1) * double3",
"parse_long(string5) * parse_long(string1)",
"parse_long(string5) * parse_long(string1) * double3"
);
private static final int ROWS_PER_SEGMENT = 100_000;
private static QueryableIndex INDEX;
private static Closer CLOSER;
@BeforeClass
public static void setupClass()
{
CLOSER = Closer.create();
final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("expression-testbench");
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.size(0)
.build();
final SegmentGenerator segmentGenerator = CLOSER.register(new SegmentGenerator());
INDEX = CLOSER.register(
segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, ROWS_PER_SEGMENT)
);
}
@AfterClass
public static void teardownClass() throws IOException
{
CLOSER.close();
}
@Parameterized.Parameters(name = "expression = {0}")
public static Iterable<?> constructorFeeder()
{
return EXPRESSIONS.stream().map(x -> new Object[]{x}).collect(Collectors.toList());
}
@Nullable
private ExprType outputType;
private String expression;
public ExpressionVectorSelectorsTest(String expression)
{
this.expression = expression;
}
@Before
public void setup()
{
Expr parsed = Parser.parse(expression, ExprMacroTable.nil());
outputType = parsed.getOutputType(
new ColumnInspector()
{
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return QueryableIndexStorageAdapter.getColumnCapabilities(INDEX, column);
}
}
);
}
@Test
public void sanityTestVectorizedExpressionSelector()
{
sanityTestVectorizedExpressionSelectors(expression, outputType, INDEX, CLOSER, ROWS_PER_SEGMENT);
}
public static void sanityTestVectorizedExpressionSelectors(
String expression,
@Nullable ExprType outputType,
QueryableIndex index,
Closer closer,
int rowsPerSegment
)
{
final List<Object> results = new ArrayList<>(rowsPerSegment);
final VirtualColumns virtualColumns = VirtualColumns.create(
ImmutableList.of(
new ExpressionVirtualColumn(
"v",
expression,
ExprType.toValueType(outputType),
TestExprMacroTable.INSTANCE
)
)
);
VectorCursor cursor = new QueryableIndexStorageAdapter(index).makeVectorCursor(
null,
index.getDataInterval(),
virtualColumns,
false,
512,
null
);
VectorValueSelector selector = null;
VectorObjectSelector objectSelector = null;
if (outputType.isNumeric()) {
selector = cursor.getColumnSelectorFactory().makeValueSelector("v");
} else {
objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector("v");
}
int rowCount = 0;
while (!cursor.isDone()) {
boolean[] nulls;
switch (outputType) {
case LONG:
nulls = selector.getNullVector();
long[] longs = selector.getLongVector();
for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) {
results.add(nulls != null && nulls[i] ? null : longs[i]);
}
break;
case DOUBLE:
nulls = selector.getNullVector();
double[] doubles = selector.getDoubleVector();
for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) {
results.add(nulls != null && nulls[i] ? null : doubles[i]);
}
break;
case STRING:
Object[] objects = objectSelector.getObjectVector();
for (int i = 0; i < objectSelector.getCurrentVectorSize(); i++, rowCount++) {
results.add(objects[i]);
}
break;
}
cursor.advance();
}
closer.register(cursor);
Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
virtualColumns,
Granularities.ALL,
false,
null
);
int rowCountCursor = cursors
.map(nonVectorized -> {
final ColumnValueSelector nonSelector = nonVectorized.getColumnSelectorFactory().makeColumnValueSelector("v");
int rows = 0;
while (!nonVectorized.isDone()) {
Assert.assertEquals(StringUtils.format("Failed at row %s", rows), nonSelector.getObject(), results.get(rows));
rows++;
nonVectorized.advance();
}
return rows;
}).accumulate(0, (acc, in) -> acc + in);
Assert.assertTrue(rowCountCursor > 0);
Assert.assertEquals(rowCountCursor, rowCount);
}
}

View File

@ -366,7 +366,7 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
SCALE_LIST_SELF_EXPLICIT.makeDimensionSelector(spec, factory);
Assert.assertTrue(selectorImplicit instanceof SingleStringInputDimensionSelector);
Assert.assertTrue(selectorExplicit instanceof MultiValueExpressionDimensionSelector);
Assert.assertTrue(selectorExplicit instanceof ExpressionMultiValueDimensionSelector);
}
@Test

View File

@ -231,6 +231,14 @@ public class QueryStackTests
final Closer closer,
final boolean useParallelMergePoolConfigured
)
{
return createQueryRunnerFactoryConglomerate(closer, getProcessingConfig(useParallelMergePoolConfigured));
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final DruidProcessingConfig processingConfig
)
{
final CloseableStupidPool<ByteBuffer> stupidPool = new CloseableStupidPool<>(
"TopNQueryRunnerFactory-bufferPool",
@ -250,7 +258,7 @@ public class QueryStackTests
return GroupByStrategySelector.STRATEGY_V2;
}
},
getProcessingConfig(useParallelMergePoolConfigured)
processingConfig
);
final GroupByQueryRunnerFactory groupByQueryRunnerFactory = factoryCloserPair.lhs;

View File

@ -316,8 +316,8 @@ public class Projection
}
// Check if a cast is necessary.
final ExprType toExprType = ExprType.fromValueType(columnValueType);
final ExprType fromExprType = ExprType.fromValueType(
final ExprType toExprType = ExprType.fromValueTypeStrict(columnValueType);
final ExprType fromExprType = ExprType.fromValueTypeStrict(
Calcites.getValueTypeForRelDataType(rexNode.getType())
);

View File

@ -136,9 +136,11 @@ public class VirtualColumnRegistry
final RowSignature.Builder builder =
RowSignature.builder().addAll(baseRowSignature);
RowSignature baseSignature = builder.build();
for (VirtualColumn virtualColumn : virtualColumnsByName.values()) {
final String columnName = virtualColumn.getOutputName();
builder.add(columnName, virtualColumn.capabilities(columnName).getType());
builder.add(columnName, virtualColumn.capabilities(baseSignature, columnName).getType());
}
return builder.build();

View File

@ -5895,9 +5895,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testExpressionFilteringAndGroupingUsingCastToLong() throws Exception
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT\n"
+ " CAST(m1 AS BIGINT) / 2 * 2,\n"
@ -6636,9 +6633,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testSumOfString() throws Exception
{
// Cannot vectorize due to expressions in aggregators.
cannotVectorize();
testQuery(
"SELECT SUM(CAST(dim1 AS INTEGER)) FROM druid.foo",
ImmutableList.of(
@ -6696,9 +6690,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() throws Exception
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT\n"
+ " FLOOR(MILLIS_TO_TIMESTAMP(cnt) TO YEAR),\n"
@ -9174,9 +9165,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Exception
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT floor(__time TO year), dim2, COUNT(*)"
+ " FROM druid.foo"
@ -11415,9 +11403,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesUsingTimeFloorWithTimestampAdd() throws Exception
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT SUM(cnt), gran FROM (\n"
+ " SELECT TIME_FLOOR(TIMESTAMPADD(DAY, -1, __time), 'P1M') AS gran,\n"
@ -11927,11 +11912,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesWithLimitAndOffset() throws Exception
{
// Cannot vectorize due to expressions.
cannotVectorize();
// Timeseries cannot handle offsets, so the query morphs into a groupBy.
testQuery(
"SELECT gran, SUM(cnt)\n"
+ "FROM (\n"
@ -11996,9 +11977,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByTimeAndOtherDimension() throws Exception
{
// Cannot vectorize due to virtual columns.
cannotVectorize();
testQuery(
"SELECT dim2, gran, SUM(cnt)\n"
+ "FROM (SELECT FLOOR(__time TO MONTH) AS gran, dim2, cnt FROM druid.foo) AS x\n"
@ -15922,8 +15900,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testRepeatedIdenticalVirtualExpressionGrouping() throws Exception
{
cannotVectorize();
final String query = "SELECT \n"
+ "\tCASE dim1 WHEN NULL THEN FALSE ELSE TRUE END AS col_a,\n"
+ "\tCASE dim2 WHEN NULL THEN FALSE ELSE TRUE END AS col_b\n"

View File

@ -0,0 +1,233 @@
/*
* 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;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.tools.RelConversionException;
import org.apache.calcite.tools.ValidationException;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@RunWith(Parameterized.class)
public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTest
{
private static final Logger log = new Logger(SqlVectorizedExpressionSanityTest.class);
private static final List<String> QUERIES = ImmutableList.of(
"SELECT SUM(long1 * long2) FROM foo",
"SELECT SUM((long1 * long2) / double1) FROM foo",
"SELECT SUM(float3 + ((long1 * long4)/double1)) FROM foo",
"SELECT SUM(long5 - (float3 + ((long1 * long4)/double1))) FROM foo",
"SELECT cos(double2) FROM foo",
"SELECT SUM(-long4) FROM foo",
"SELECT SUM(PARSE_LONG(string1)) FROM foo",
"SELECT SUM(PARSE_LONG(string3)) FROM foo",
"SELECT TIME_FLOOR(__time, 'PT1H'), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 1",
"SELECT TIME_FLOOR(__time, 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 2",
"SELECT TIME_FLOOR(TIMESTAMPADD(DAY, -1, __time), 'PT1H'), SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 1",
"SELECT (long1 * long2), SUM(double1) FROM foo GROUP BY 1 ORDER BY 2",
"SELECT string2, SUM(long1 * long4) FROM foo GROUP BY 1 ORDER BY 2"
);
private static final int ROWS_PER_SEGMENT = 100_000;
private static QueryableIndex INDEX;
private static Closer CLOSER;
private static QueryRunnerFactoryConglomerate CONGLOMERATE;
private static SpecificSegmentsQuerySegmentWalker WALKER;
@Nullable
private static PlannerFactory PLANNER_FACTORY;
@BeforeClass
public static void setupClass()
{
CLOSER = Closer.create();
final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("expression-testbench");
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.size(0)
.build();
final SegmentGenerator segmentGenerator = CLOSER.register(new SegmentGenerator());
INDEX = CLOSER.register(
segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, ROWS_PER_SEGMENT)
);
CONGLOMERATE = QueryStackTests.createQueryRunnerFactoryConglomerate(CLOSER);
WALKER = new SpecificSegmentsQuerySegmentWalker(CONGLOMERATE).add(
dataSegment,
INDEX
);
CLOSER.register(WALKER);
final PlannerConfig plannerConfig = new PlannerConfig();
final SchemaPlus rootSchema =
CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
PLANNER_FACTORY = new PlannerFactory(
rootSchema,
CalciteTests.createMockQueryLifecycleFactory(WALKER, CONGLOMERATE),
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME
);
}
@AfterClass
public static void teardownClass() throws IOException
{
CLOSER.close();
}
@Parameterized.Parameters(name = "query = {0}")
public static Iterable<?> constructorFeeder()
{
return QUERIES.stream().map(x -> new Object[]{x}).collect(Collectors.toList());
}
private String query;
public SqlVectorizedExpressionSanityTest(String query)
{
this.query = query;
}
@Test
public void testQuery() throws SqlParseException, RelConversionException, ValidationException
{
sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query);
}
public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query)
throws ValidationException, RelConversionException, SqlParseException
{
final Map<String, Object> vector = ImmutableMap.of("vectorize", true);
final Map<String, Object> nonvector = ImmutableMap.of("vectorize", false);
final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
.createEscalatedAuthenticationResult();
try (
final DruidPlanner vectorPlanner = plannerFactory.createPlanner(vector, ImmutableList.of(), authenticationResult);
final DruidPlanner nonVectorPlanner = plannerFactory.createPlanner(nonvector, ImmutableList.of(), authenticationResult)
) {
final PlannerResult vectorPlan = vectorPlanner.plan(query);
final PlannerResult nonVectorPlan = nonVectorPlanner.plan(query);
final Sequence<Object[]> vectorSequence = vectorPlan.run();
final Sequence<Object[]> nonVectorSequence = nonVectorPlan.run();
Yielder<Object[]> vectorizedYielder = Yielders.each(vectorSequence);
Yielder<Object[]> nonVectorizedYielder = Yielders.each(nonVectorSequence);
int row = 0;
int misMatch = 0;
while (!vectorizedYielder.isDone() && !nonVectorizedYielder.isDone()) {
Object[] vectorGet = vectorizedYielder.get();
Object[] nonVectorizedGet = nonVectorizedYielder.get();
try {
Assert.assertEquals(vectorGet.length, nonVectorizedGet.length);
for (int i = 0; i < vectorGet.length; i++) {
Object nonVectorObject = nonVectorizedGet[i];
Object vectorObject = vectorGet[i];
if (vectorObject instanceof Float || vectorObject instanceof Double) {
Assert.assertEquals(
StringUtils.format(
"Double results differed at row %s (%s : %s)",
row,
nonVectorObject,
vectorObject
),
((Double) nonVectorObject).doubleValue(),
((Double) vectorObject).doubleValue(),
0.01
);
} else {
Assert.assertEquals(
StringUtils.format(
"Results differed at row %s (%s : %s)",
row,
nonVectorObject,
vectorObject
),
nonVectorObject,
vectorObject
);
}
}
}
catch (Throwable t) {
log.warn(t.getMessage());
misMatch++;
}
vectorizedYielder = vectorizedYielder.next(vectorGet);
nonVectorizedYielder = nonVectorizedYielder.next(nonVectorizedGet);
row++;
}
Assert.assertEquals("Expected no mismatched results", 0, misMatch);
Assert.assertTrue(vectorizedYielder.isDone());
Assert.assertTrue(nonVectorizedYielder.isDone());
}
}
}