ExpressionSelectors: Add optimized selectors. (#5048)

* ExpressionSelectors: Add caching selectors.

- SingleLongInputCaching selector for expressions on the __time column,
  using a similar optimization to SingleScanTimeDimSelector
- SingleStringInputDimensionSelector for expressions on string columns
  that return strings, using a similar optimization to ExtractionFn
  based DimensionSelectors.
- SingleStringInputCaching selector for expressions on string columns
  that return primitives.

Also, in the SQL planner, prefer expressions for time operations
rather than extractionFns.

* Code review comments.
This commit is contained in:
Gian Merlino 2017-11-13 20:24:24 -08:00 committed by Fangjin Yang
parent 4fd4444b42
commit 77df5e0673
25 changed files with 1415 additions and 409 deletions

View File

@ -69,7 +69,7 @@ import java.util.function.Function;
@Measurement(iterations = 30)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class ExpressionBenchmark
public class ExpressionAggregationBenchmark
{
@Param({"1000000"})
private int rowsPerSegment;
@ -101,7 +101,7 @@ public class ExpressionBenchmark
.build();
this.segmentGenerator = new SegmentGenerator();
this.index = segmentGenerator.generate(dataSegment, schemaInfo, rowsPerSegment);
this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
this.javaScriptAggregatorFactory = new JavaScriptAggregatorFactory(
"name",
ImmutableList.of("x", "y"),

View File

@ -0,0 +1,374 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.benchmark;
import com.google.common.collect.ImmutableList;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.SegmentGenerator;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.expression.TestExprMacroTable;
import io.druid.query.extraction.StrlenExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexStorageAdapter;
import io.druid.segment.VirtualColumns;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import io.druid.segment.virtual.ExpressionVirtualColumn;
import io.druid.timeline.DataSegment;
import io.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 java.util.ArrayList;
import java.util.BitSet;
import java.util.List;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 15)
@Measurement(iterations = 30)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class ExpressionSelectorBenchmark
{
@Param({"1000000"})
private int rowsPerSegment;
private SegmentGenerator segmentGenerator;
private QueryableIndex index;
@Setup(Level.Trial)
public void setup() throws Exception
{
final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
ImmutableList.of(
BenchmarkColumnSchema.makeNormal("n", ValueType.LONG, false, 1, 0d, 0d, 10000d, false),
BenchmarkColumnSchema.makeZipf(
"s",
ValueType.STRING,
false,
1,
0d,
1000,
10000,
3d
)
),
ImmutableList.of(),
Intervals.of("2000/P1D"),
false
);
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build();
this.segmentGenerator = new SegmentGenerator();
this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment);
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
if (index != null) {
index.close();
index = null;
}
if (segmentGenerator != null) {
segmentGenerator.close();
segmentGenerator = null;
}
}
@Benchmark
public void timeFloorUsingExpression(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.create(
ImmutableList.of(
new ExpressionVirtualColumn(
"v",
"timestamp_floor(__time, 'PT1H')",
ValueType.LONG,
TestExprMacroTable.INSTANCE
)
)
),
Granularities.ALL,
false,
null
);
final List<?> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
while (!cursor.isDone()) {
blackhole.consume(selector.getLong());
cursor.advance();
}
return null;
}
),
new ArrayList<>()
);
blackhole.consume(results);
}
@Benchmark
public void timeFloorUsingExtractionFn(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.EMPTY,
Granularities.ALL,
false,
null
);
final List<?> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
final DimensionSelector selector = cursor
.getColumnSelectorFactory()
.makeDimensionSelector(
new ExtractionDimensionSpec(
Column.TIME_COLUMN_NAME,
"v",
new TimeFormatExtractionFn(null, null, null, Granularities.HOUR, true)
)
);
consumeDimension(cursor, selector, blackhole);
return null;
}
),
new ArrayList<>()
);
blackhole.consume(results);
}
@Benchmark
public void timeFloorUsingCursor(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.EMPTY,
Granularities.HOUR,
false,
null
);
final List<Long> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
long count = 0L;
while (!cursor.isDone()) {
count++;
cursor.advance();
}
return count;
}
),
new ArrayList<>()
);
long count = 0L;
for (Long result : results) {
count += result;
}
blackhole.consume(count);
}
@Benchmark
public void strlenUsingExpressionAsLong(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.create(
ImmutableList.of(
new ExpressionVirtualColumn(
"v",
"strlen(s)",
ValueType.STRING,
TestExprMacroTable.INSTANCE
)
)
),
Granularities.ALL,
false,
null
);
final List<?> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
return null;
}
),
new ArrayList<>()
);
blackhole.consume(results);
}
@Benchmark
public void strlenUsingExpressionAsString(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.create(
ImmutableList.of(
new ExpressionVirtualColumn(
"v",
"strlen(s)",
ValueType.STRING,
TestExprMacroTable.INSTANCE
)
)
),
Granularities.ALL,
false,
null
);
final List<?> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(
new DefaultDimensionSpec("v", "v", ValueType.STRING)
);
consumeDimension(cursor, selector, blackhole);
return null;
}
),
new ArrayList<>()
);
blackhole.consume(results);
}
@Benchmark
public void strlenUsingExtractionFn(Blackhole blackhole) throws Exception
{
final Sequence<Cursor> cursors = new QueryableIndexStorageAdapter(index).makeCursors(
null,
index.getDataInterval(),
VirtualColumns.EMPTY,
Granularities.ALL,
false,
null
);
final List<?> results = Sequences.toList(
Sequences.map(
cursors,
cursor -> {
final DimensionSelector selector = cursor
.getColumnSelectorFactory()
.makeDimensionSelector(new ExtractionDimensionSpec("x", "v", StrlenExtractionFn.instance()));
consumeDimension(cursor, selector, blackhole);
return null;
}
),
new ArrayList<>()
);
blackhole.consume(results);
}
private void consumeDimension(final Cursor cursor, final DimensionSelector selector, final Blackhole blackhole)
{
if (selector.getValueCardinality() >= 0) {
// Read all IDs and then lookup all names.
final BitSet values = new BitSet();
while (!cursor.isDone()) {
final int value = selector.getRow().get(0);
values.set(value);
cursor.advance();
}
for (int i = values.nextSetBit(0); i >= 0; i = values.nextSetBit(i + 1)) {
blackhole.consume(selector.lookupName(i));
}
} else {
// Lookup names as we go.
while (!cursor.isDone()) {
final int value = selector.getRow().get(0);
blackhole.consume(selector.lookupName(value));
cursor.advance();
}
}
}
private void consumeLong(final Cursor cursor, final ColumnValueSelector selector, final Blackhole blackhole)
{
while (!cursor.isDone()) {
blackhole.consume(selector.getLong());
cursor.advance();
}
}
}

View File

@ -31,6 +31,7 @@ import io.druid.data.input.impl.LongDimensionSchema;
import io.druid.data.input.impl.StringDimensionSchema;
import io.druid.hll.HyperLogLogHash;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
@ -71,6 +72,7 @@ public class SegmentGenerator implements Closeable
public QueryableIndex generate(
final DataSegment dataSegment,
final BenchmarkSchemaInfo schemaInfo,
final Granularity granularity,
final int numRows
)
{
@ -109,6 +111,7 @@ public class SegmentGenerator implements Closeable
.withDimensionsSpec(new DimensionsSpec(dimensions, ImmutableList.of(), ImmutableList.of()))
.withMetrics(schemaInfo.getAggsArray())
.withRollup(schemaInfo.isWithRollup())
.withQueryGranularity(granularity)
.build();
final List<InputRow> rows = new ArrayList<>();

View File

@ -109,7 +109,7 @@ public class SqlBenchmark
this.segmentGenerator = new SegmentGenerator();
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, rowsPerSegment);
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final QueryRunnerFactoryConglomerate conglomerate = CalciteTests.queryRunnerFactoryConglomerate();
final PlannerConfig plannerConfig = new PlannerConfig();

View File

@ -31,10 +31,11 @@ import io.druid.segment.BaseDoubleColumnValueSelector;
import io.druid.segment.BaseFloatColumnValueSelector;
import io.druid.segment.BaseLongColumnValueSelector;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DoubleColumnSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.virtual.ExpressionObjectSelector;
import io.druid.segment.virtual.ExpressionSelectors;
import java.util.HashSet;
import java.util.LinkedList;
@ -140,7 +141,7 @@ public class AggregatorUtil
}
if (fieldName == null && fieldExpression != null) {
final Expr expr = Parser.parse(fieldExpression, macroTable);
final ExpressionObjectSelector baseSelector = ExpressionObjectSelector.from(metricFactory, expr);
final ColumnValueSelector<ExprEval> baseSelector = ExpressionSelectors.makeExprEvalSelector(metricFactory, expr);
class ExpressionFloatColumnSelector implements FloatColumnSelector
{
@Override
@ -174,7 +175,7 @@ public class AggregatorUtil
}
if (fieldName == null && fieldExpression != null) {
final Expr expr = Parser.parse(fieldExpression, macroTable);
final ExpressionObjectSelector baseSelector = ExpressionObjectSelector.from(metricFactory, expr);
final ColumnValueSelector<ExprEval> baseSelector = ExpressionSelectors.makeExprEvalSelector(metricFactory, expr);
class ExpressionLongColumnSelector implements LongColumnSelector
{
@Override
@ -208,7 +209,7 @@ public class AggregatorUtil
}
if (fieldName == null && fieldExpression != null) {
final Expr expr = Parser.parse(fieldExpression, macroTable);
final ExpressionObjectSelector baseSelector = ExpressionObjectSelector.from(metricFactory, expr);
final ColumnValueSelector<ExprEval> baseSelector = ExpressionSelectors.makeExprEvalSelector(metricFactory, expr);
class ExpressionDoubleColumnSelector implements DoubleColumnSelector
{
@Override

View File

@ -0,0 +1,89 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment;
import com.google.common.base.Preconditions;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import javax.annotation.Nullable;
public class ConstantColumnValueSelector<T> implements ColumnValueSelector<T>
{
private long longValue;
private float floatValue;
private double doubleValue;
@Nullable
private T objectValue;
private Class<T> objectClass;
public ConstantColumnValueSelector(
final long longValue,
final float floatValue,
final double doubleValue,
@Nullable final T objectValue,
final Class<T> objectClass
)
{
this.longValue = longValue;
this.floatValue = floatValue;
this.doubleValue = doubleValue;
this.objectValue = objectValue;
this.objectClass = Preconditions.checkNotNull(objectClass, "objectClass");
}
@Override
public double getDouble()
{
return doubleValue;
}
@Override
public float getFloat()
{
return floatValue;
}
@Override
public long getLong()
{
return longValue;
}
@Nullable
@Override
public T getObject()
{
return objectValue;
}
@Override
public Class<T> classOfObject()
{
return objectClass;
}
@Override
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
{
// Nothing here: objectValue is nullable but getObject is not @CalledFromHotLoop
}
}

View File

@ -96,6 +96,7 @@ public interface DimensionSelector extends ColumnValueSelector, HotLoopCallee
* @return the field name for the given id
*/
@CalledFromHotLoop
@Nullable
String lookupName(int id);
/**

View File

@ -61,7 +61,7 @@ public class SingleScanTimeDimSelector implements DimensionSelector
@Override
public IndexedInts getRow()
{
return new SingleIndexedInt(getDimensionValueIndex());
return SingleIndexedInt.of(getDimensionValueIndex());
}
@Override

View File

@ -205,7 +205,7 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn<St
@Override
public IndexedInts getRow()
{
return new SingleIndexedInt(getRowValue());
return SingleIndexedInt.of(getRowValue());
}
public int getRowValue()
@ -216,7 +216,7 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn<St
@Override
public IndexedInts getRow(int offset)
{
return new SingleIndexedInt(getRowValue(offset));
return SingleIndexedInt.of(getRowValue(offset));
}
@Override

View File

@ -25,13 +25,31 @@ import java.io.IOException;
public final class SingleIndexedInt implements IndexedInts
{
private static final int CACHE_SIZE = 128;
private static final SingleIndexedInt[] CACHE = new SingleIndexedInt[CACHE_SIZE];
static {
for (int i = 0; i < CACHE_SIZE; i++) {
CACHE[i] = new SingleIndexedInt(i);
}
}
private final int value;
public SingleIndexedInt(int value)
private SingleIndexedInt(int value)
{
this.value = value;
}
public static SingleIndexedInt of(int value)
{
if (value >= 0 && value < CACHE_SIZE) {
return CACHE[value];
} else {
return new SingleIndexedInt(value);
}
}
@Override
public int size()
{

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import io.druid.math.expr.Evals;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.Parser;
import io.druid.query.BitmapResultFactory;
import io.druid.query.expression.ExprUtils;
@ -51,7 +52,7 @@ public class ExpressionFilter implements Filter
@Override
public ValueMatcher makeMatcher(final ColumnSelectorFactory factory)
{
final ColumnValueSelector selector = ExpressionSelectors.makeColumnValueSelector(factory, expr);
final ColumnValueSelector<ExprEval> selector = ExpressionSelectors.makeExprEvalSelector(factory, expr);
return new ValueMatcher()
{
@Override

View File

@ -0,0 +1,78 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import com.google.common.base.Preconditions;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.ColumnValueSelector;
import javax.annotation.Nonnull;
public class ExpressionColumnValueSelector implements ColumnValueSelector<ExprEval>
{
private final Expr.ObjectBinding bindings;
private final Expr expression;
public ExpressionColumnValueSelector(Expr expression, Expr.ObjectBinding bindings)
{
this.bindings = Preconditions.checkNotNull(bindings, "bindings");
this.expression = Preconditions.checkNotNull(expression, "expression");
}
@Override
public double getDouble()
{
return getObject().asDouble();
}
@Override
public float getFloat()
{
return (float) getObject().asDouble();
}
@Override
public long getLong()
{
return getObject().asLong();
}
@Override
public Class<ExprEval> classOfObject()
{
return ExprEval.class;
}
@Nonnull
@Override
public ExprEval getObject()
{
return expression.eval(bindings);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("expression", expression);
inspector.visit("bindings", bindings);
}
}

View File

@ -1,159 +0,0 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.collect.Maps;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.Parser;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.BaseObjectColumnValueSelector;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.Map;
public class ExpressionObjectSelector implements ObjectColumnSelector<ExprEval>
{
private final Expr expression;
private final Expr.ObjectBinding bindings;
private ExpressionObjectSelector(Expr.ObjectBinding bindings, Expr expression)
{
this.bindings = Preconditions.checkNotNull(bindings, "bindings");
this.expression = Preconditions.checkNotNull(expression, "expression");
}
public static ExpressionObjectSelector from(ColumnSelectorFactory columnSelectorFactory, Expr expression)
{
return new ExpressionObjectSelector(createBindings(columnSelectorFactory, expression), expression);
}
private static Expr.ObjectBinding createBindings(ColumnSelectorFactory columnSelectorFactory, Expr expression)
{
final Map<String, Supplier<Object>> suppliers = Maps.newHashMap();
for (String columnName : Parser.findRequiredBindings(expression)) {
final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(columnName);
final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null;
final Supplier<Object> supplier;
if (nativeType == ValueType.FLOAT) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getFloat;
} else if (nativeType == ValueType.LONG) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getLong;
} else if (nativeType == ValueType.DOUBLE) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getDouble;
} else if (nativeType == ValueType.STRING) {
supplier = supplierFromDimensionSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName))
);
} else if (nativeType == null) {
// Unknown ValueType. Try making an Object selector and see if that gives us anything useful.
supplier = supplierFromObjectSelector(columnSelectorFactory.makeColumnValueSelector(columnName));
} else {
// Unhandleable ValueType (COMPLEX).
supplier = null;
}
if (supplier != null) {
suppliers.put(columnName, supplier);
}
}
return Parser.withSuppliers(suppliers);
}
@VisibleForTesting
@Nonnull
static Supplier<Object> supplierFromDimensionSelector(final DimensionSelector selector)
{
Preconditions.checkNotNull(selector, "selector");
return () -> {
final IndexedInts row = selector.getRow();
if (row.size() == 0) {
// Treat empty multi-value rows as nulls.
return null;
} else if (row.size() == 1) {
return selector.lookupName(row.get(0));
} else {
// Can't handle multi-value rows in expressions.
// Treat them as nulls until we think of something better to do.
return null;
}
};
}
@VisibleForTesting
@Nullable
static Supplier<Object> supplierFromObjectSelector(final BaseObjectColumnValueSelector<?> selector)
{
if (selector == null) {
return null;
}
final Class<?> clazz = selector.classOfObject();
if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) {
// Number, String supported as-is.
return selector::getObject;
} else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) {
// Might be Numbers and Strings. Use a selector that double-checks.
return () -> {
final Object val = selector.getObject();
if (val instanceof Number || val instanceof String) {
return val;
} else {
return null;
}
};
} else {
// No numbers or strings.
return null;
}
}
@Override
public Class<ExprEval> classOfObject()
{
return ExprEval.class;
}
@Override
public ExprEval getObject()
{
return expression.eval(bindings);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("expression", expression);
inspector.visit("bindings", bindings);
}
}

View File

@ -19,16 +19,35 @@
package io.druid.segment.virtual;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.base.Supplier;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.Parser;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.expression.ExprUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.BaseObjectColumnValueSelector;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.ConstantColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.DimensionSelectorUtils;
import io.druid.segment.NilColumnValueSelector;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
public class ExpressionSelectors
{
@ -37,41 +56,46 @@ public class ExpressionSelectors
// No instantiation.
}
/**
* Makes a ColumnValueSelector whose getObject method returns an Object that is the value computed by
* an {@link ExprEval}.
*
* @see ExpressionSelectors#makeExprEvalSelector(ColumnSelectorFactory, Expr)
*/
public static ColumnValueSelector makeColumnValueSelector(
ColumnSelectorFactory columnSelectorFactory,
Expr expression
)
{
final ExpressionObjectSelector baseSelector = ExpressionObjectSelector.from(columnSelectorFactory, expression);
final ColumnValueSelector<ExprEval> baseSelector = makeExprEvalSelector(columnSelectorFactory, expression);
return new ColumnValueSelector()
{
@Override
public double getDouble()
{
final ExprEval exprEval = baseSelector.getObject();
return exprEval.isNull() ? 0.0 : exprEval.asDouble();
return baseSelector.getDouble();
}
@Override
public float getFloat()
{
final ExprEval exprEval = baseSelector.getObject();
return exprEval.isNull() ? 0.0f : (float) exprEval.asDouble();
return baseSelector.getFloat();
}
@Override
public long getLong()
{
final ExprEval exprEval = baseSelector.getObject();
return exprEval.isNull() ? 0L : exprEval.asLong();
return baseSelector.getLong();
}
@Nullable
@Override
public Object getObject()
{
final ExprEval exprEval = baseSelector.getObject();
return exprEval.value();
// No need for null check on getObject() since baseSelector impls will never return null.
//noinspection ConstantConditions
return baseSelector.getObject().value();
}
@Override
@ -88,15 +112,87 @@ public class ExpressionSelectors
};
}
/**
* Makes a ColumnValueSelector whose getObject method returns an {@link ExprEval}.
*
* @see ExpressionSelectors#makeColumnValueSelector(ColumnSelectorFactory, Expr)
*/
public static ColumnValueSelector<ExprEval> makeExprEvalSelector(
ColumnSelectorFactory columnSelectorFactory,
Expr expression
)
{
final List<String> columns = Parser.findRequiredBindings(expression);
if (columns.size() == 1) {
final String column = Iterables.getOnlyElement(columns);
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
if (column.equals(Column.TIME_COLUMN_NAME)) {
// Optimization for expressions that hit the __time column and nothing else.
// May be worth applying this optimization to all long columns?
return new SingleLongInputCachingExpressionColumnValueSelector(
columnSelectorFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
expression
);
} else if (capabilities != null
&& capabilities.getType() == ValueType.STRING
&& capabilities.isDictionaryEncoded()) {
// Optimization for expressions that hit one string column and nothing else.
return new SingleStringInputCachingExpressionColumnValueSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ValueType.STRING)),
expression
);
}
}
final Expr.ObjectBinding bindings = createBindings(expression, columnSelectorFactory);
if (bindings.equals(ExprUtils.nilBindings())) {
// Optimization for constant expressions.
final ExprEval eval = expression.eval(bindings);
return new ConstantColumnValueSelector<>(
eval.asLong(),
(float) eval.asDouble(),
eval.asDouble(),
eval,
ExprEval.class
);
}
// No special optimization.
return new ExpressionColumnValueSelector(expression, bindings);
}
public static DimensionSelector makeDimensionSelector(
final ColumnSelectorFactory columnSelectorFactory,
final Expr expression,
final ExtractionFn extractionFn
)
{
final ExpressionObjectSelector baseSelector = ExpressionObjectSelector.from(columnSelectorFactory, expression);
final List<String> columns = Parser.findRequiredBindings(expression);
if (extractionFn == null) {
if (columns.size() == 1) {
final String column = Iterables.getOnlyElement(columns);
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
if (capabilities != null
&& capabilities.getType() == ValueType.STRING
&& capabilities.isDictionaryEncoded()) {
// Optimization for dimension selectors that wrap a single underlying string column.
return new SingleStringInputDimensionSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ValueType.STRING)),
expression
);
}
}
final ColumnValueSelector<ExprEval> baseSelector = makeExprEvalSelector(columnSelectorFactory, expression);
if (baseSelector instanceof ConstantColumnValueSelector) {
// Optimization for dimension selectors on constants.
return DimensionSelectorUtils.constantSelector(baseSelector.getObject().asString(), extractionFn);
} else if (extractionFn == null) {
class DefaultExpressionDimensionSelector extends BaseSingleValueDimensionSelector
{
@Override
@ -131,4 +227,97 @@ public class ExpressionSelectors
return new ExtractionExpressionDimensionSelector();
}
}
private static Expr.ObjectBinding createBindings(Expr expression, ColumnSelectorFactory columnSelectorFactory)
{
final Map<String, Supplier<Object>> suppliers = Maps.newHashMap();
for (String columnName : Parser.findRequiredBindings(expression)) {
final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(columnName);
final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null;
final Supplier<Object> supplier;
if (nativeType == ValueType.FLOAT) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getFloat;
} else if (nativeType == ValueType.LONG) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getLong;
} else if (nativeType == ValueType.DOUBLE) {
supplier = columnSelectorFactory.makeColumnValueSelector(columnName)::getDouble;
} else if (nativeType == ValueType.STRING) {
supplier = supplierFromDimensionSelector(
columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName))
);
} else if (nativeType == null) {
// Unknown ValueType. Try making an Object selector and see if that gives us anything useful.
supplier = supplierFromObjectSelector(columnSelectorFactory.makeColumnValueSelector(columnName));
} else {
// Unhandleable ValueType (COMPLEX).
supplier = null;
}
if (supplier != null) {
suppliers.put(columnName, supplier);
}
}
if (suppliers.isEmpty()) {
return ExprUtils.nilBindings();
} else if (suppliers.size() == 1) {
// If there's only one supplier, we can skip the Map and just use that supplier when asked for something.
final String column = Iterables.getOnlyElement(suppliers.keySet());
final Supplier<Object> supplier = Iterables.getOnlyElement(suppliers.values());
return identifierName -> {
// There's only one binding, and it must be the single column, so it can safely be ignored in production.
assert column.equals(identifierName);
return supplier.get();
};
} else {
return Parser.withSuppliers(suppliers);
}
}
@VisibleForTesting
@Nonnull
static Supplier<Object> supplierFromDimensionSelector(final DimensionSelector selector)
{
Preconditions.checkNotNull(selector, "selector");
return () -> {
final IndexedInts row = selector.getRow();
if (row.size() == 1) {
return selector.lookupName(row.get(0));
} else {
// Can't handle non-singly-valued rows in expressions.
// Treat them as nulls until we think of something better to do.
return null;
}
};
}
@Nullable
static Supplier<Object> supplierFromObjectSelector(final BaseObjectColumnValueSelector<?> selector)
{
if (selector instanceof NilColumnValueSelector) {
return null;
}
final Class<?> clazz = selector.classOfObject();
if (Number.class.isAssignableFrom(clazz) || String.class.isAssignableFrom(clazz)) {
// Number, String supported as-is.
return selector::getObject;
} else if (clazz.isAssignableFrom(Number.class) || clazz.isAssignableFrom(String.class)) {
// Might be Numbers and Strings. Use a selector that double-checks.
return () -> {
final Object val = selector.getObject();
if (val instanceof Number || val instanceof String) {
return val;
} else {
return null;
}
};
} else {
// No numbers or strings.
return null;
}
}
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import io.druid.math.expr.Expr;
import javax.annotation.Nullable;
public class SingleInputBindings implements Expr.ObjectBinding
{
private Object value;
@Override
public Object get(final String name)
{
return value;
}
public void set(@Nullable final Object value)
{
this.value = value;
}
}

View File

@ -0,0 +1,146 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.Parser;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.ColumnValueSelector;
import javax.annotation.Nonnull;
/**
* Like {@link ExpressionColumnValueSelector}, but caches the most recently computed value and re-uses it in the case
* of runs in the underlying column. This is especially useful for the __time column, where we expect runs.
*/
public class SingleLongInputCachingExpressionColumnValueSelector implements ColumnValueSelector<ExprEval>
{
enum Validity
{
NONE,
DOUBLE,
LONG,
EVAL
}
private final ColumnValueSelector selector;
private final Expr expression;
private final SingleInputBindings bindings = new SingleInputBindings();
// Last read input value
private long lastInput;
// Last computed output values (validity determined by "validity" field)
private Validity validity = Validity.NONE;
private double lastDoubleOutput;
private long lastLongOutput;
private ExprEval lastEvalOutput;
public SingleLongInputCachingExpressionColumnValueSelector(
final ColumnValueSelector selector,
final Expr expression
)
{
// Verify expression has just one binding.
if (Parser.findRequiredBindings(expression).size() != 1) {
throw new ISE("WTF?! Expected expression with just one binding");
}
this.selector = Preconditions.checkNotNull(selector, "selector");
this.expression = Preconditions.checkNotNull(expression, "expression");
}
@Override
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
inspector.visit("expression", expression);
}
@Override
public double getDouble()
{
final long currentInput = selector.getLong();
if (lastInput == currentInput && validity == Validity.DOUBLE) {
return lastDoubleOutput;
} else {
final double output = eval(currentInput).asDouble();
lastInput = currentInput;
lastDoubleOutput = output;
validity = Validity.DOUBLE;
return output;
}
}
@Override
public float getFloat()
{
return (float) getDouble();
}
@Override
public long getLong()
{
final long currentInput = selector.getLong();
if (lastInput == currentInput && validity == Validity.LONG) {
return lastLongOutput;
} else {
final long output = eval(currentInput).asLong();
lastInput = currentInput;
lastLongOutput = output;
validity = Validity.LONG;
return output;
}
}
@Nonnull
@Override
public ExprEval getObject()
{
final long currentInput = selector.getLong();
if (lastInput == currentInput && validity == Validity.EVAL) {
return lastEvalOutput;
} else {
final ExprEval output = eval(currentInput);
lastInput = currentInput;
lastEvalOutput = output;
validity = Validity.EVAL;
return output;
}
}
@Override
public Class<ExprEval> classOfObject()
{
return ExprEval.class;
}
private ExprEval eval(final long value)
{
bindings.set(value);
return expression.eval(bindings);
}
}

View File

@ -0,0 +1,164 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.Expr;
import io.druid.math.expr.ExprEval;
import io.druid.math.expr.Parser;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.IndexedInts;
import it.unimi.dsi.fastutil.ints.Int2ObjectLinkedOpenHashMap;
import javax.annotation.Nullable;
/**
* Like {@link ExpressionColumnValueSelector}, but caches results for the first CACHE_SIZE dictionary IDs of
* a string column. Must only be used on selectors with dictionaries.
*/
public class SingleStringInputCachingExpressionColumnValueSelector implements ColumnValueSelector<ExprEval>
{
private static final int CACHE_SIZE = 1000;
private final DimensionSelector selector;
private final Expr expression;
private final Expr.ObjectBinding bindings;
private final ExprEval[] arrayEvalCache;
private final LruEvalCache lruEvalCache;
public SingleStringInputCachingExpressionColumnValueSelector(
final DimensionSelector selector,
final Expr expression
)
{
// Verify expression has just one binding.
if (Parser.findRequiredBindings(expression).size() != 1) {
throw new ISE("WTF?! Expected expression with just one binding");
}
this.selector = Preconditions.checkNotNull(selector, "selector");
this.expression = Preconditions.checkNotNull(expression, "expression");
final Supplier<Object> inputSupplier = ExpressionSelectors.supplierFromDimensionSelector(selector);
this.bindings = name -> inputSupplier.get();
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
throw new ISE("Selector must have a dictionary");
} else if (selector.getValueCardinality() <= CACHE_SIZE) {
arrayEvalCache = new ExprEval[selector.getValueCardinality()];
lruEvalCache = null;
} else {
arrayEvalCache = null;
lruEvalCache = new LruEvalCache(expression, bindings);
}
}
@Override
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
inspector.visit("expression", expression);
}
@Override
public double getDouble()
{
return eval().asDouble();
}
@Override
public float getFloat()
{
return (float) eval().asDouble();
}
@Override
public long getLong()
{
return eval().asLong();
}
@Nullable
@Override
public ExprEval getObject()
{
return eval();
}
@Override
public Class<ExprEval> classOfObject()
{
return ExprEval.class;
}
private ExprEval eval()
{
final IndexedInts row = selector.getRow();
if (row.size() == 1) {
final int id = row.get(0);
if (arrayEvalCache != null) {
if (arrayEvalCache[id] == null) {
arrayEvalCache[id] = expression.eval(bindings);
}
return arrayEvalCache[id];
} else {
assert lruEvalCache != null;
return lruEvalCache.compute(id);
}
}
return expression.eval(bindings);
}
public static class LruEvalCache
{
private final Expr expression;
private final Expr.ObjectBinding bindings;
private final Int2ObjectLinkedOpenHashMap<ExprEval> m = new Int2ObjectLinkedOpenHashMap<>(CACHE_SIZE);
public LruEvalCache(final Expr expression, final Expr.ObjectBinding bindings)
{
this.expression = expression;
this.bindings = bindings;
}
public ExprEval compute(final int id)
{
ExprEval value = m.getAndMoveToFirst(id);
if (value == null) {
value = expression.eval(bindings);
m.putAndMoveToFirst(id, value);
if (m.size() > CACHE_SIZE) {
m.removeLast();
}
}
return value;
}
}
}

View File

@ -0,0 +1,161 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.virtual;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import io.druid.java.util.common.ISE;
import io.druid.math.expr.Expr;
import io.druid.math.expr.Parser;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.DimensionSelectorUtils;
import io.druid.segment.IdLookup;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.SingleIndexedInt;
import javax.annotation.Nullable;
/**
* A DimensionSelector decorator that computes an expression on top of it.
*/
public class SingleStringInputDimensionSelector implements DimensionSelector
{
private final DimensionSelector selector;
private final Expr expression;
private final SingleInputBindings bindings = new SingleInputBindings();
/**
* 0 if selector has null as a value; 1 if it doesn't.
*/
private final int nullAdjustment;
public SingleStringInputDimensionSelector(
final DimensionSelector selector,
final Expr expression
)
{
// Verify expression has just one binding.
if (Parser.findRequiredBindings(expression).size() != 1) {
throw new ISE("WTF?! Expected expression with just one binding");
}
// Verify selector has a working dictionary.
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN
|| !selector.nameLookupPossibleInAdvance()) {
throw new ISE("Selector of class[%s] does not have a dictionary, cannot use it.", selector.getClass().getName());
}
this.selector = Preconditions.checkNotNull(selector, "selector");
this.expression = Preconditions.checkNotNull(expression, "expression");
this.nullAdjustment = selector.getValueCardinality() == 0 || selector.lookupName(0) != null ? 1 : 0;
}
@Override
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
inspector.visit("expression", expression);
}
/**
* Treats any non-single-valued row as a row containing a single null value, to ensure consistency with
* other expression selectors. See also {@link ExpressionSelectors#supplierFromDimensionSelector} for similar
* behavior.
*/
@Override
public IndexedInts getRow()
{
final IndexedInts row = selector.getRow();
if (row.size() == 1) {
if (nullAdjustment == 0) {
return row;
} else {
return SingleIndexedInt.of(row.get(0) + nullAdjustment);
}
} else {
// Can't handle non-singly-valued rows in expressions.
// Treat them as nulls until we think of something better to do.
return SingleIndexedInt.of(0);
}
}
@Override
public ValueMatcher makeValueMatcher(@Nullable final String value)
{
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
}
@Override
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
{
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
}
@Override
public int getValueCardinality()
{
return selector.getValueCardinality() + nullAdjustment;
}
@Override
public String lookupName(final int id)
{
final String value;
if (id == 0) {
// id 0 is always null for this selector impl.
value = null;
} else {
value = selector.lookupName(id - nullAdjustment);
}
bindings.set(value);
return expression.eval(bindings).asString();
}
@Override
public boolean nameLookupPossibleInAdvance()
{
return true;
}
@Nullable
@Override
public IdLookup idLookup()
{
return null;
}
@Nullable
@Override
public Object getObject()
{
return defaultGetObject();
}
@Override
public Class classOfObject()
{
return Object.class;
}
}

View File

@ -583,11 +583,6 @@ public class GroupByQueryRunnerTest
.setGranularity(QueryRunnerTestHelper.dayGran)
.build();
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
expectedException.expect(UnsupportedOperationException.class);
expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality.");
}
List<Row> expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotivex", "rows", 1L, "idx", 135L),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "businessx", "rows", 1L, "idx", 118L),

View File

@ -30,13 +30,13 @@ import org.junit.Test;
import java.util.List;
public class ExpressionObjectSelectorTest
public class ExpressionColumnValueSelectorTest
{
@Test
public void testSupplierFromDimensionSelector()
{
final SettableSupplier<String> settableSupplier = new SettableSupplier<>();
final Supplier<Object> supplier = ExpressionObjectSelector.supplierFromDimensionSelector(
final Supplier<Object> supplier = ExpressionSelectors.supplierFromDimensionSelector(
dimensionSelectorFromSupplier(settableSupplier)
);
@ -54,7 +54,7 @@ public class ExpressionObjectSelectorTest
public void testSupplierFromObjectSelectorObject()
{
final SettableSupplier<Object> settableSupplier = new SettableSupplier<>();
final Supplier<Object> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
final Supplier<Object> supplier = ExpressionSelectors.supplierFromObjectSelector(
objectSelectorFromSupplier(settableSupplier, Object.class)
);
@ -78,7 +78,7 @@ public class ExpressionObjectSelectorTest
public void testSupplierFromObjectSelectorNumber()
{
final SettableSupplier<Number> settableSupplier = new SettableSupplier<>();
final Supplier<Object> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
final Supplier<Object> supplier = ExpressionSelectors.supplierFromObjectSelector(
objectSelectorFromSupplier(settableSupplier, Number.class)
);
@ -97,7 +97,7 @@ public class ExpressionObjectSelectorTest
public void testSupplierFromObjectSelectorString()
{
final SettableSupplier<String> settableSupplier = new SettableSupplier<>();
final Supplier<Object> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
final Supplier<Object> supplier = ExpressionSelectors.supplierFromObjectSelector(
objectSelectorFromSupplier(settableSupplier, String.class)
);
@ -115,7 +115,7 @@ public class ExpressionObjectSelectorTest
public void testSupplierFromObjectSelectorList()
{
final SettableSupplier<List> settableSupplier = new SettableSupplier<>();
final Supplier<Object> supplier = ExpressionObjectSelector.supplierFromObjectSelector(
final Supplier<Object> supplier = ExpressionSelectors.supplierFromObjectSelector(
objectSelectorFromSupplier(settableSupplier, List.class)
);

View File

@ -24,15 +24,19 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.Row;
import io.druid.java.util.common.DateTimes;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.expression.TestExprMacroTable;
import io.druid.query.extraction.BucketExtractionFn;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
import io.druid.query.groupby.RowBasedColumnSelectorFactory;
import io.druid.segment.BaseFloatColumnValueSelector;
import io.druid.segment.BaseLongColumnValueSelector;
import io.druid.segment.BaseObjectColumnValueSelector;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import org.junit.Assert;
@ -41,24 +45,24 @@ import org.junit.Test;
public class ExpressionVirtualColumnTest
{
private static final InputRow ROW0 = new MapBasedInputRow(
0,
DateTimes.of("2000-01-01T00:00:00").getMillis(),
ImmutableList.of(),
ImmutableMap.of()
);
private static final InputRow ROW1 = new MapBasedInputRow(
0,
DateTimes.of("2000-01-01T00:00:00").getMillis(),
ImmutableList.of(),
ImmutableMap.of("x", 4)
);
private static final InputRow ROW2 = new MapBasedInputRow(
0,
DateTimes.of("2000-01-01T02:00:00").getMillis(),
ImmutableList.of(),
ImmutableMap.of("x", 2.1, "y", 3L, "z", "foobar")
);
private static final InputRow ROW3 = new MapBasedInputRow(
0,
DateTimes.of("2000-01-02T01:00:00").getMillis(),
ImmutableList.of(),
ImmutableMap.of("x", 2L, "y", 3L, "z", "foobar")
);
@ -87,23 +91,34 @@ public class ExpressionVirtualColumnTest
ValueType.STRING,
TestExprMacroTable.INSTANCE
);
private static final TestColumnSelectorFactory COLUMN_SELECTOR_FACTORY = new TestColumnSelectorFactory();
private static final ExpressionVirtualColumn TIMEFLOOR = new ExpressionVirtualColumn(
"expr",
"timestamp_floor(__time, 'P1D')",
ValueType.LONG,
TestExprMacroTable.INSTANCE
);
private static final ThreadLocal<Row> CURRENT_ROW = new ThreadLocal<>();
private static final ColumnSelectorFactory COLUMN_SELECTOR_FACTORY = RowBasedColumnSelectorFactory.create(
CURRENT_ROW,
null
);
@Test
public void testObjectSelector()
{
final BaseObjectColumnValueSelector selector = XPLUSY.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(null, selector.getObject());
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(4.0d, selector.getObject());
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(5.1d, selector.getObject());
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(5L, selector.getObject());
}
@ -112,16 +127,16 @@ public class ExpressionVirtualColumnTest
{
final BaseLongColumnValueSelector selector = XPLUSY.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(0L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(4L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(5L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(5L, selector.getLong());
}
@ -130,16 +145,16 @@ public class ExpressionVirtualColumnTest
{
final BaseLongColumnValueSelector selector = ZCONCATX.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(0L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(4L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(0L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(0L, selector.getLong());
}
@ -148,16 +163,16 @@ public class ExpressionVirtualColumnTest
{
final BaseFloatColumnValueSelector selector = XPLUSY.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(0.0f, selector.getFloat(), 0.0f);
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(4.0f, selector.getFloat(), 0.0f);
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(5.1f, selector.getFloat(), 0.0f);
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(5.0f, selector.getFloat(), 0.0f);
}
@ -173,25 +188,25 @@ public class ExpressionVirtualColumnTest
final ValueMatcher fiveMatcher = selector.makeValueMatcher("5");
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(Predicates.<String>notNull());
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(true, nullMatcher.matches());
Assert.assertEquals(false, fiveMatcher.matches());
Assert.assertEquals(false, nonNullMatcher.matches());
Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(false, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
Assert.assertEquals("4.0", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(false, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
Assert.assertEquals("5.1", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(true, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
@ -208,19 +223,19 @@ public class ExpressionVirtualColumnTest
Assert.assertNotNull(selector);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(1, selector.getRow().size());
Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(1, selector.getRow().size());
Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(1, selector.getRow().size());
Assert.assertEquals("foobar2.1", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(1, selector.getRow().size());
Assert.assertEquals("foobar2", selector.lookupName(selector.getRow().get(0)));
}
@ -237,25 +252,25 @@ public class ExpressionVirtualColumnTest
final ValueMatcher fiveMatcher = selector.makeValueMatcher("5");
final ValueMatcher nonNullMatcher = selector.makeValueMatcher(Predicates.<String>notNull());
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(true, nullMatcher.matches());
Assert.assertEquals(false, fiveMatcher.matches());
Assert.assertEquals(false, nonNullMatcher.matches());
Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(false, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(true, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
Assert.assertEquals("5", selector.lookupName(selector.getRow().get(0)));
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(false, nullMatcher.matches());
Assert.assertEquals(true, fiveMatcher.matches());
Assert.assertEquals(true, nonNullMatcher.matches());
@ -268,28 +283,50 @@ public class ExpressionVirtualColumnTest
final BaseLongColumnValueSelector selector =
CONSTANT_LIKE.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(1L, selector.getLong());
}
@Test
public void testLongSelectorWithZLikeExprMacro()
{
final BaseLongColumnValueSelector selector = ZLIKE.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
final ColumnValueSelector selector = ZLIKE.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
COLUMN_SELECTOR_FACTORY.setRow(ROW0);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(0L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW1);
CURRENT_ROW.set(ROW1);
Assert.assertEquals(0L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW2);
CURRENT_ROW.set(ROW2);
Assert.assertEquals(1L, selector.getLong());
COLUMN_SELECTOR_FACTORY.setRow(ROW3);
CURRENT_ROW.set(ROW3);
Assert.assertEquals(1L, selector.getLong());
}
@Test
public void testLongSelectorOfTimeColumn()
{
final ColumnValueSelector selector = TIMEFLOOR.makeColumnValueSelector("expr", COLUMN_SELECTOR_FACTORY);
CURRENT_ROW.set(ROW0);
Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getLong());
Assert.assertEquals((float) DateTimes.of("2000-01-01").getMillis(), selector.getFloat(), 0.0f);
Assert.assertEquals((double) DateTimes.of("2000-01-01").getMillis(), selector.getDouble(), 0.0d);
Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getObject());
CURRENT_ROW.set(ROW1);
Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getLong());
CURRENT_ROW.set(ROW2);
Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getLong());
CURRENT_ROW.set(ROW3);
Assert.assertEquals(DateTimes.of("2000-01-02").getMillis(), selector.getLong());
Assert.assertEquals(DateTimes.of("2000-01-02").getMillis(), selector.getDouble(), 0.0);
}
@Test
public void testRequiredColumns()
{

View File

@ -19,7 +19,7 @@
package io.druid.sql.calcite.expression.builtin;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.StringUtils;
import io.druid.query.expression.TimestampExtractExprMacro;
import io.druid.sql.calcite.expression.DruidExpression;
@ -37,8 +37,6 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.joda.time.DateTimeZone;
import java.util.Map;
public class TimeExtractOperatorConversion implements SqlOperatorConversion
{
private static final SqlFunction SQL_FUNCTION = OperatorConversions
@ -49,44 +47,18 @@ public class TimeExtractOperatorConversion implements SqlOperatorConversion
.functionCategory(SqlFunctionCategory.TIMEDATE)
.build();
// Note that QUARTER is not supported here.
private static final Map<TimestampExtractExprMacro.Unit, String> EXTRACT_FORMAT_MAP =
ImmutableMap.<TimestampExtractExprMacro.Unit, String>builder()
.put(TimestampExtractExprMacro.Unit.SECOND, "s")
.put(TimestampExtractExprMacro.Unit.MINUTE, "m")
.put(TimestampExtractExprMacro.Unit.HOUR, "H")
.put(TimestampExtractExprMacro.Unit.DAY, "d")
.put(TimestampExtractExprMacro.Unit.DOW, "e")
.put(TimestampExtractExprMacro.Unit.DOY, "D")
.put(TimestampExtractExprMacro.Unit.WEEK, "w")
.put(TimestampExtractExprMacro.Unit.MONTH, "M")
.put(TimestampExtractExprMacro.Unit.YEAR, "Y")
.build();
public static DruidExpression applyTimeExtract(
final DruidExpression timeExpression,
final TimestampExtractExprMacro.Unit unit,
final DateTimeZone timeZone
)
{
return timeExpression.map(
simpleExtraction -> {
final String formatString = EXTRACT_FORMAT_MAP.get(unit);
if (formatString == null) {
return null;
} else {
return TimeFormatOperatorConversion.applyTimestampFormat(
simpleExtraction,
formatString,
timeZone
);
}
},
expression -> StringUtils.format(
"timestamp_extract(%s,%s,%s)",
expression,
DruidExpression.stringLiteral(unit.name()),
DruidExpression.stringLiteral(timeZone.getID())
return DruidExpression.fromFunctionCall(
"timestamp_extract",
ImmutableList.of(
timeExpression,
DruidExpression.fromExpression(DruidExpression.stringLiteral(unit.name())),
DruidExpression.fromExpression(DruidExpression.stringLiteral(timeZone.getID()))
)
);
}

View File

@ -19,16 +19,10 @@
package io.druid.sql.calcite.expression.builtin;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.sql.calcite.expression.DruidExpression;
import io.druid.sql.calcite.expression.Expressions;
import io.druid.sql.calcite.expression.ExtractionFns;
import io.druid.sql.calcite.expression.OperatorConversions;
import io.druid.sql.calcite.expression.SimpleExtraction;
import io.druid.sql.calcite.expression.SqlOperatorConversion;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
@ -54,33 +48,6 @@ public class TimeFormatOperatorConversion implements SqlOperatorConversion
.functionCategory(SqlFunctionCategory.TIMEDATE)
.build();
public static SimpleExtraction applyTimestampFormat(
final SimpleExtraction simpleExtraction,
final String pattern,
final DateTimeZone timeZone
)
{
Preconditions.checkNotNull(simpleExtraction, "simpleExtraction");
Preconditions.checkNotNull(pattern, "pattern");
Preconditions.checkNotNull(timeZone, "timeZone");
final ExtractionFn baseExtractionFn = simpleExtraction.getExtractionFn();
if (baseExtractionFn instanceof TimeFormatExtractionFn) {
final TimeFormatExtractionFn baseTimeFormatFn = (TimeFormatExtractionFn) baseExtractionFn;
final Granularity queryGranularity = ExtractionFns.toQueryGranularity(baseTimeFormatFn);
if (queryGranularity != null) {
// Combine EXTRACT(X FROM FLOOR(Y TO Z)) into a single extractionFn.
return SimpleExtraction.of(
simpleExtraction.getColumn(),
new TimeFormatExtractionFn(pattern, timeZone, null, queryGranularity, true)
);
}
}
return simpleExtraction.cascade(new TimeFormatExtractionFn(pattern, timeZone, null, null, true));
}
@Override
public SqlOperator calciteOperator()
{
@ -108,16 +75,13 @@ public class TimeFormatOperatorConversion implements SqlOperatorConversion
? DateTimeZone.forID(RexLiteral.stringValue(call.getOperands().get(2)))
: plannerContext.getTimeZone();
return timeExpression.map(
simpleExtraction -> applyTimestampFormat(simpleExtraction, pattern, timeZone),
expression -> DruidExpression.functionCall(
return DruidExpression.fromFunctionCall(
"timestamp_format",
ImmutableList.of(
expression,
timeExpression.getExpression(),
DruidExpression.stringLiteral(pattern),
DruidExpression.stringLiteral(timeZone.getID())
).stream().map(DruidExpression::fromExpression).collect(Collectors.toList())
)
);
}
}

View File

@ -5565,22 +5565,14 @@ public class CalciteQueryTest
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
DIMS(
new ExtractionDimensionSpec(
"__time",
"d0",
ValueType.LONG,
new TimeFormatExtractionFn(
"Y",
DateTimeZone.UTC,
null,
Granularities.NONE,
true
)
)
.setVirtualColumns(
EXPRESSION_VIRTUAL_COLUMN(
"d0:v",
"timestamp_extract(\"__time\",'YEAR','UTC')",
ValueType.LONG
)
)
.setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
@ -5619,22 +5611,14 @@ public class CalciteQueryTest
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
DIMS(
new ExtractionDimensionSpec(
"__time",
"d0",
ValueType.STRING,
new TimeFormatExtractionFn(
"yyyy MM",
DateTimeZone.UTC,
null,
Granularities.NONE,
true
)
)
.setVirtualColumns(
EXPRESSION_VIRTUAL_COLUMN(
"d0:v",
"timestamp_format(\"__time\",'yyyy MM','UTC')",
ValueType.STRING
)
)
.setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.STRING)))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setLimitSpec(
new DefaultLimitSpec(
@ -5671,22 +5655,14 @@ public class CalciteQueryTest
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
DIMS(
new ExtractionDimensionSpec(
"__time",
"d0",
ValueType.LONG,
new TimeFormatExtractionFn(
"Y",
DateTimeZone.UTC,
null,
Granularities.YEAR,
true
)
)
.setVirtualColumns(
EXPRESSION_VIRTUAL_COLUMN(
"d0:v",
"timestamp_extract(timestamp_floor(\"__time\",'P1Y','','UTC'),'YEAR','UTC')",
ValueType.LONG
)
)
.setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
@ -5714,22 +5690,14 @@ public class CalciteQueryTest
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(QSS(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
DIMS(
new ExtractionDimensionSpec(
"__time",
"d0",
ValueType.LONG,
new TimeFormatExtractionFn(
"Y",
DateTimeZone.forID(LOS_ANGELES),
null,
new PeriodGranularity(Period.years(1), null, DateTimeZone.forID(LOS_ANGELES)),
true
)
)
.setVirtualColumns(
EXPRESSION_VIRTUAL_COLUMN(
"d0:v",
"timestamp_extract(timestamp_floor(\"__time\",'P1Y','','America/Los_Angeles'),'YEAR','America/Los_Angeles')",
ValueType.LONG
)
)
.setDimensions(DIMS(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG)))
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_LOS_ANGELES)
.build()

View File

@ -482,10 +482,7 @@ public class ExpressionsTest
inputRef("t"),
rexBuilder.makeLiteral("QUARTER")
),
DruidExpression.of(
null,
"timestamp_extract(\"t\",'QUARTER','UTC')"
),
DruidExpression.fromExpression("timestamp_extract(\"t\",'QUARTER','UTC')"),
1L
);
@ -496,13 +493,7 @@ public class ExpressionsTest
rexBuilder.makeLiteral("DAY"),
rexBuilder.makeLiteral("America/Los_Angeles")
),
DruidExpression.of(
SimpleExtraction.of(
"t",
new TimeFormatExtractionFn("d", LOS_ANGELES, null, Granularities.NONE, true)
),
"timestamp_extract(\"t\",'DAY','America/Los_Angeles')"
),
DruidExpression.fromExpression("timestamp_extract(\"t\",'DAY','America/Los_Angeles')"),
2L
);
}
@ -635,13 +626,7 @@ public class ExpressionsTest
inputRef("t"),
rexBuilder.makeLiteral("yyyy-MM-dd HH:mm:ss")
),
DruidExpression.of(
SimpleExtraction.of(
"t",
new TimeFormatExtractionFn("yyyy-MM-dd HH:mm:ss", DateTimeZone.UTC, null, Granularities.NONE, true)
),
"timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss','UTC')"
),
DruidExpression.fromExpression("timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss','UTC')"),
"2000-02-03 04:05:06"
);
@ -652,19 +637,7 @@ public class ExpressionsTest
rexBuilder.makeLiteral("yyyy-MM-dd HH:mm:ss"),
rexBuilder.makeLiteral("America/Los_Angeles")
),
DruidExpression.of(
SimpleExtraction.of(
"t",
new TimeFormatExtractionFn(
"yyyy-MM-dd HH:mm:ss",
LOS_ANGELES,
null,
Granularities.NONE,
true
)
),
"timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss','America/Los_Angeles')"
),
DruidExpression.fromExpression("timestamp_format(\"t\",'yyyy-MM-dd HH:mm:ss','America/Los_Angeles')"),
"2000-02-02 20:05:06"
);
}
@ -678,10 +651,7 @@ public class ExpressionsTest
rexBuilder.makeFlag(TimeUnitRange.QUARTER),
inputRef("t")
),
DruidExpression.of(
null,
"timestamp_extract(\"t\",'QUARTER','UTC')"
),
DruidExpression.fromExpression("timestamp_extract(\"t\",'QUARTER','UTC')"),
1L
);
@ -691,13 +661,7 @@ public class ExpressionsTest
rexBuilder.makeFlag(TimeUnitRange.DAY),
inputRef("t")
),
DruidExpression.of(
SimpleExtraction.of(
"t",
new TimeFormatExtractionFn("d", DateTimeZone.UTC, null, Granularities.NONE, true)
),
"timestamp_extract(\"t\",'DAY','UTC')"
),
DruidExpression.fromExpression("timestamp_extract(\"t\",'DAY','UTC')"),
3L
);
}