未验证 提交 9362dc79 编写于 作者: C Clint Wylie 提交者: GitHub

re-use expression vector evaluation results for the same offset in expression...

re-use expression vector evaluation results for the same offset in expression vector selectors (#10614)

* cache expression selector results by associating vector expression bindings to underlying vector offset

* better coverage, fix floats

* style

* stupid bot

* stupid me

* more test

* intellij threw me under the bus when it generated those junit methods

* narrow interface instead of passing around offset
上级 2fc2938b
......@@ -265,6 +265,13 @@ public interface Expr
boolean[] getNullVector(String name);
int getCurrentVectorSize();
/**
* Returns an integer that uniquely identifies the current position of the underlying vector offset, if this
* binding is backed by a segment. This is useful for caching: it is safe to assume nothing has changed in the
* offset so long as the id remains the same. See also: ReadableVectorOffset (in druid-processing)
*/
int getCurrentVectorId();
}
/**
......
......@@ -412,6 +412,8 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
private final int vectorSize;
private int id = 0;
SettableVectorInputBinding(int vectorSize)
{
this.nulls = new HashMap<>();
......@@ -504,5 +506,12 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
{
return vectorSize;
}
@Override
public int getCurrentVectorId()
{
// never cache, this is just for tests anyway
return id++;
}
}
}
......@@ -24,6 +24,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.ObjectColumnSelector;
import org.apache.druid.segment.data.ReadableOffset;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.VectorObjectSelector;
......@@ -106,7 +107,7 @@ public interface ComplexColumn extends BaseColumn
{
final Object[] vector = new Object[offset.getMaxVectorSize()];
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
@Override
public Object[] getObjectVector()
......
......@@ -38,6 +38,7 @@ import org.apache.druid.segment.filter.BooleanValueMatcher;
import org.apache.druid.segment.historical.HistoricalDimensionSelector;
import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorObjectSelector;
......@@ -336,7 +337,7 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<St
class QueryableSingleValueDimensionVectorSelector implements SingleValueDimensionVectorSelector, IdLookup
{
private final int[] vector = new int[offset.getMaxVectorSize()];
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
@Override
public int[] getRowVector()
......@@ -409,7 +410,7 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<St
class QueryableMultiValueDimensionVectorSelector implements MultiValueDimensionVectorSelector, IdLookup
{
private final IndexedInts[] vector = new IndexedInts[offset.getMaxVectorSize()];
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
@Override
public IndexedInts[] getRowVector()
......@@ -496,7 +497,7 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<St
{
private final int[] vector = new int[offset.getMaxVectorSize()];
private final String[] strings = new String[offset.getMaxVectorSize()];
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
@Override
......
......@@ -26,6 +26,7 @@ import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DoubleColumnSelector;
import org.apache.druid.segment.historical.HistoricalColumnSelector;
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.VectorSelectorUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
......@@ -154,7 +155,7 @@ public interface ColumnarDoubles extends Closeable
{
private final double[] doubleVector;
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
private int offsetMark = -1;
......
......@@ -26,6 +26,7 @@ import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.FloatColumnSelector;
import org.apache.druid.segment.historical.HistoricalColumnSelector;
import org.apache.druid.segment.vector.BaseFloatVectorValueSelector;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.VectorSelectorUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
......@@ -154,7 +155,7 @@ public interface ColumnarFloats extends Closeable
{
private final float[] floatVector;
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
private int offsetMark = -1;
......
......@@ -26,6 +26,7 @@ import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.LongColumnSelector;
import org.apache.druid.segment.historical.HistoricalColumnSelector;
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.ReadableVectorOffset;
import org.apache.druid.segment.vector.VectorSelectorUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
......@@ -154,7 +155,7 @@ public interface ColumnarLongs extends Closeable
{
private final long[] longVector;
private int id = ReadableVectorOffset.NULL_ID;
private int id = ReadableVectorInspector.NULL_ID;
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
private int offsetMark = -1;
......
......@@ -67,7 +67,7 @@ public class FalseFilter implements Filter
@Override
public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
{
return new FalseVectorMatcher(factory.getVectorSizeInspector());
return new FalseVectorMatcher(factory.getReadableVectorInspector());
}
@Override
......
......@@ -63,7 +63,7 @@ public class TrueFilter implements Filter
@Override
public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
{
return new TrueVectorMatcher(factory.getVectorSizeInspector());
return new TrueVectorMatcher(factory.getReadableVectorInspector());
}
@Override
......
......@@ -23,8 +23,8 @@ public abstract class BaseDoubleVectorValueSelector implements VectorValueSelect
{
protected final ReadableVectorOffset offset;
private int longId = ReadableVectorOffset.NULL_ID;
private int floatId = ReadableVectorOffset.NULL_ID;
private int longId = ReadableVectorInspector.NULL_ID;
private int floatId = ReadableVectorInspector.NULL_ID;
private long[] longVector;
private float[] floatVector;
......
......@@ -23,8 +23,8 @@ public abstract class BaseFloatVectorValueSelector implements VectorValueSelecto
{
protected final ReadableVectorOffset offset;
private int longId = ReadableVectorOffset.NULL_ID;
private int doubleId = ReadableVectorOffset.NULL_ID;
private int longId = ReadableVectorInspector.NULL_ID;
private int doubleId = ReadableVectorInspector.NULL_ID;
private long[] longVector;
private double[] doubleVector;
......
......@@ -23,8 +23,8 @@ public abstract class BaseLongVectorValueSelector implements VectorValueSelector
{
protected final ReadableVectorOffset offset;
private int floatId = ReadableVectorOffset.NULL_ID;
private int doubleId = ReadableVectorOffset.NULL_ID;
private int floatId = ReadableVectorInspector.NULL_ID;
private int doubleId = ReadableVectorInspector.NULL_ID;
private float[] floatVector;
private double[] doubleVector;
......
......@@ -71,7 +71,7 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
}
@Override
public VectorSizeInspector getVectorSizeInspector()
public ReadableVectorInspector getReadableVectorInspector()
{
return offset;
}
......
/*
* 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.vector;
/**
* Vector inspector that can supply a unique identifier of the vector to use with caching in addition to
* sizing information
*/
public interface ReadableVectorInspector extends VectorSizeInspector
{
/**
* A marker value that will never be returned by "getId".
*/
int NULL_ID = -1;
/**
* Returns an integer that uniquely identifies the current vector. This is useful for caching: it is safe to assume
* nothing has changed in the vector so long as the id remains the same.
*/
int getId();
}
......@@ -28,20 +28,8 @@ package org.apache.druid.segment.vector;
* @see VectorOffset, the movable version.
* @see org.apache.druid.segment.data.ReadableOffset, the non-vectorized version.
*/
public interface ReadableVectorOffset extends VectorSizeInspector
public interface ReadableVectorOffset extends ReadableVectorInspector
{
/**
* A marker value that will never be returned by "getId".
*/
int NULL_ID = -1;
/**
* Returns an integer that uniquely identifies the current position of the offset. Should *not* be construed as an
* actual offset; for that, use "getStartOffset" or "getOffsets". This is useful for caching: it is safe to assume
* nothing has changed in the offset so long as the id remains the same.
*/
int getId();
/**
* Checks if the current batch is a contiguous range or not. This is only good for one batch at a time, since the
* same object may return some contiguous batches and some non-contiguous batches. So, callers must check this method
......
......@@ -33,9 +33,9 @@ import javax.annotation.Nullable;
public interface VectorColumnSelectorFactory extends ColumnInspector
{
/**
* Returns a {@link VectorSizeInspector} for the {@link VectorCursor} that generated this object.
* Returns a {@link ReadableVectorInspector} for the {@link VectorCursor} that generated this object.
*/
VectorSizeInspector getVectorSizeInspector();
ReadableVectorInspector getReadableVectorInspector();
/**
* Returns the maximum vector size for the {@link VectorCursor} that generated this object.
......@@ -44,7 +44,7 @@ public interface VectorColumnSelectorFactory extends ColumnInspector
*/
default int getMaxVectorSize()
{
return getVectorSizeInspector().getMaxVectorSize();
return getReadableVectorInspector().getMaxVectorSize();
}
/**
......
......@@ -22,8 +22,8 @@ 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.ReadableVectorInspector;
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;
......@@ -37,15 +37,15 @@ class ExpressionVectorInputBinding implements Expr.VectorInputBinding
private final Map<String, ExprType> types;
private final NilVectorSelector nilSelector;
private final VectorSizeInspector sizeInspector;
private final ReadableVectorInspector vectorInspector;
public ExpressionVectorInputBinding(VectorSizeInspector sizeInspector)
public ExpressionVectorInputBinding(ReadableVectorInspector vectorInspector)
{
this.numeric = new HashMap<>();
this.objects = new HashMap<>();
this.types = new HashMap<>();
this.sizeInspector = sizeInspector;
this.nilSelector = NilVectorSelector.create(sizeInspector);
this.vectorInspector = vectorInspector;
this.nilSelector = NilVectorSelector.create(this.vectorInspector);
}
public ExpressionVectorInputBinding addNumeric(String name, ExprType type, VectorValueSelector selector)
......@@ -63,15 +63,15 @@ class ExpressionVectorInputBinding implements Expr.VectorInputBinding
}
@Override
public <T> T[] getObjectVector(String name)
public ExprType getType(String name)
{
return (T[]) objects.getOrDefault(name, nilSelector).getObjectVector();
return types.get(name);
}
@Override
public ExprType getType(String name)
public <T> T[] getObjectVector(String name)
{
return types.get(name);
return (T[]) objects.getOrDefault(name, nilSelector).getObjectVector();
}
@Override
......@@ -96,12 +96,18 @@ class ExpressionVectorInputBinding implements Expr.VectorInputBinding
@Override
public int getMaxVectorSize()
{
return sizeInspector.getMaxVectorSize();
return vectorInspector.getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return sizeInspector.getCurrentVectorSize();
return vectorInspector.getCurrentVectorSize();
}
@Override
public int getCurrentVectorId()
{
return vectorInspector.getId();
}
}
......@@ -22,12 +22,18 @@ 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.ReadableVectorInspector;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
public class ExpressionVectorObjectSelector implements VectorObjectSelector
{
final Expr.VectorInputBinding bindings;
final ExprVectorProcessor<?> processor;
private final Expr.VectorInputBinding bindings;
private final ExprVectorProcessor<?> processor;
@MonotonicNonNull
private Object[] cached;
private int currentId = ReadableVectorInspector.NULL_ID;
public ExpressionVectorObjectSelector(ExprVectorProcessor<?> processor, Expr.VectorInputBinding bindings)
{
......@@ -38,7 +44,11 @@ public class ExpressionVectorObjectSelector implements VectorObjectSelector
@Override
public Object[] getObjectVector()
{
return processor.evalVector(bindings).getObjectVector();
if (bindings.getCurrentVectorId() != currentId) {
currentId = bindings.getCurrentVectorId();
cached = processor.evalVector(bindings).getObjectVector();
}
return cached;
}
@Override
......
......@@ -52,7 +52,7 @@ public class ExpressionVectorSelectors
if (plan.isConstant()) {
String constant = plan.getExpression().eval(ExprUtils.nilBindings()).asString();
return ConstantVectorSelectors.singleValueDimensionVectorSelector(factory.getVectorSizeInspector(), constant);
return ConstantVectorSelectors.singleValueDimensionVectorSelector(factory.getReadableVectorInspector(), constant);
}
throw new IllegalStateException("Only constant expressions currently support dimension selectors");
}
......@@ -67,7 +67,7 @@ public class ExpressionVectorSelectors
if (plan.isConstant()) {
return ConstantVectorSelectors.vectorValueSelector(
factory.getVectorSizeInspector(),
factory.getReadableVectorInspector(),
(Number) plan.getExpression().eval(ExprUtils.nilBindings()).value()
);
}
......@@ -86,7 +86,7 @@ public class ExpressionVectorSelectors
if (plan.isConstant()) {
return ConstantVectorSelectors.vectorObjectSelector(
factory.getVectorSizeInspector(),
factory.getReadableVectorInspector(),
plan.getExpression().eval(ExprUtils.nilBindings()).value()
);
}
......@@ -101,7 +101,9 @@ public class ExpressionVectorSelectors
VectorColumnSelectorFactory vectorColumnSelectorFactory
)
{
ExpressionVectorInputBinding binding = new ExpressionVectorInputBinding(vectorColumnSelectorFactory.getVectorSizeInspector());
ExpressionVectorInputBinding binding = new ExpressionVectorInputBinding(
vectorColumnSelectorFactory.getReadableVectorInspector()
);
final List<String> columns = bindingAnalysis.getRequiredBindingsList();
for (String columnName : columns) {
final ColumnCapabilities columnCapabilities = vectorColumnSelectorFactory.getColumnCapabilities(columnName);
......
......@@ -21,16 +21,24 @@ 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.ExprEvalVector;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.apache.druid.segment.vector.ReadableVectorInspector;
import org.apache.druid.segment.vector.VectorValueSelector;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import javax.annotation.Nullable;
public class ExpressionVectorValueSelector implements VectorValueSelector
{
final Expr.VectorInputBinding bindings;
final ExprVectorProcessor<?> processor;
final float[] floats;
private final Expr.VectorInputBinding bindings;
private final ExprVectorProcessor<?> processor;
private final float[] floats;
@MonotonicNonNull
private ExprEvalVector<?> evalResult;
private int currentOffsetId = ReadableVectorInspector.NULL_ID;
private int currentFloatsId = ReadableVectorInspector.NULL_ID;
public ExpressionVectorValueSelector(ExprVectorProcessor<?> processor, Expr.VectorInputBinding bindings)
{
......@@ -42,15 +50,20 @@ public class ExpressionVectorValueSelector implements VectorValueSelector
@Override
public long[] getLongVector()
{
return processor.evalVector(bindings).getLongVector();
eval();
return evalResult.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];
if (currentFloatsId != bindings.getCurrentVectorId()) {
eval();
currentFloatsId = currentOffsetId;
final double[] doubles = evalResult.getDoubleVector();
for (int i = 0; i < bindings.getCurrentVectorSize(); i++) {
floats[i] = (float) doubles[i];
}
}
return floats;
}
......@@ -58,14 +71,16 @@ public class ExpressionVectorValueSelector implements VectorValueSelector
@Override
public double[] getDoubleVector()
{
return processor.evalVector(bindings).getDoubleVector();
eval();
return evalResult.getDoubleVector();
}
@Nullable
@Override
public boolean[] getNullVector()
{
return processor.evalVector(bindings).getNullVector();
eval();
return evalResult.getNullVector();
}
@Override
......@@ -79,4 +94,12 @@ public class ExpressionVectorValueSelector implements VectorValueSelector
{
return bindings.getCurrentVectorSize();
}
private void eval()
{
if (currentOffsetId != bindings.getCurrentVectorId()) {
currentOffsetId = bindings.getCurrentVectorId();
evalResult = processor.evalVector(bindings);
}
}
}
......@@ -109,7 +109,7 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
Assert.assertEquals(outputName, dimensionSpec.getOutputName());
return new SingleValueDimensionVectorSelector()
{
private final VectorSizeInspector inspector = factory.getVectorSizeInspector();
private final VectorSizeInspector inspector = factory.getReadableVectorInspector();
private final int[] rowVector = new int[inspector.getMaxVectorSize()];
@Override
......@@ -166,11 +166,11 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
)
{
Assert.assertEquals(outputName, dimensionSpec.getOutputName());
final IndexedInts[] rowVector = new IndexedInts[factory.getVectorSizeInspector().getMaxVectorSize()];
final IndexedInts[] rowVector = new IndexedInts[factory.getReadableVectorInspector().getMaxVectorSize()];
Arrays.fill(rowVector, new ArrayBasedIndexedInts(new int[]{0, 0}));
return new MultiValueDimensionVectorSelector()
{
private final VectorSizeInspector inspector = factory.getVectorSizeInspector();
private final VectorSizeInspector inspector = factory.getReadableVectorInspector();
@Override
public IndexedInts[] getRowVector()
......@@ -225,9 +225,9 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
)
{
Assert.assertEquals(outputName, columnName);
final long[] longs = new long[factory.getVectorSizeInspector().getMaxVectorSize()];
final double[] doubles = new double[factory.getVectorSizeInspector().getMaxVectorSize()];
final float[] floats = new float[factory.getVectorSizeInspector().getMaxVectorSize()];
final long[] longs = new long[factory.getReadableVectorInspector().getMaxVectorSize()];
final double[] doubles = new double[factory.getReadableVectorInspector().getMaxVectorSize()];
final float[] floats = new float[factory.getReadableVectorInspector().getMaxVectorSize()];
Arrays.fill(longs, 2L);
Arrays.fill(doubles, 2.0);
Arrays.fill(floats, 2.0f);
......@@ -261,13 +261,13 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
@Override
public int getMaxVectorSize()
{
return factory.getVectorSizeInspector().getMaxVectorSize();
return factory.getReadableVectorInspector().getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return factory.getVectorSizeInspector().getCurrentVectorSize();
return factory.getReadableVectorInspector().getCurrentVectorSize();
}
};
}
......@@ -279,7 +279,7 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
)
{
Assert.assertEquals(outputName, columnName);
final Object[] objects = new Object[factory.getVectorSizeInspector().getMaxVectorSize()];
final Object[] objects = new Object[factory.getReadableVectorInspector().getMaxVectorSize()];
if (capabilities.hasMultipleValues().isTrue()) {
Arrays.fill(objects, ImmutableList.of("2", "2"));
} else {
......@@ -290,13 +290,13 @@ public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
@Override
public int getMaxVectorSize()
{
return factory.getVectorSizeInspector().getMaxVectorSize();
return factory.getReadableVectorInspector().getMaxVectorSize();
}
@Override
public int getCurrentVectorSize()
{
return factory.getVectorSizeInspector().getCurrentVectorSize();
return factory.getReadableVectorInspector().getCurrentVectorSize();
}
@Override
......
/*
* 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.vector.ExprEvalStringVector;
import org.apache.druid.math.expr.vector.ExprEvalVector;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class ExpressionVectorObjectSelectorTest
{
private static final int MAX_SIZE = 8;
private Expr.VectorInputBinding binding;
private ExprVectorProcessor vectorProcessor;
private ExpressionVectorObjectSelector expressionVectorValueSelector;
@Before
public void setUp()
{
binding = EasyMock.createMock(Expr.VectorInputBinding.class);
vectorProcessor = EasyMock.createMock(ExprVectorProcessor.class);
EasyMock.expect(binding.getMaxVectorSize()).andReturn(MAX_SIZE).once();
EasyMock.replay(binding, vectorProcessor);
expressionVectorValueSelector = new ExpressionVectorObjectSelector(vectorProcessor, binding);
EasyMock.reset(binding, vectorProcessor);
}
@After
public void tearDown()
{
EasyMock.verify(binding, vectorProcessor);
}
@Test
public void testSelectObject()
{
final String[] vector = new String[]{"1", "2", null, "3"};
ExprEvalVector vectorEval = new ExprEvalStringVector(vector);
EasyMock.expect(binding.getCurrentVectorId()).andReturn(1).anyTimes();
EasyMock.expect(vectorProcessor.evalVector(binding)).andReturn(vectorEval).once();
EasyMock.replay(binding, vectorProcessor);
Object[] vector1 = expressionVectorValueSelector.getObjectVector();
Object[] vector2 = expressionVectorValueSelector.getObjectVector();
Assert.assertArrayEquals(vector1, vector2);
}
}
......@@ -84,7 +84,11 @@ public class ExpressionVectorSelectorsTest
"'string constant'",
"1",
"192412.24124",
"null"
"null",
"long2",
"float2",
"double2",
"string3"
);
private static final int ROWS_PER_SEGMENT = 100_000;
......@@ -219,10 +223,19 @@ public class ExpressionVectorSelectorsTest
}
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]);
// special case to test floats just to get coverage on getFloatVector
if ("float2".equals(expression)) {
nulls = selector.getNullVector();
float[] floats = selector.getFloatVector();
for (int i = 0; i < selector.getCurrentVectorSize(); i++, rowCount++) {
results.add(nulls != null && nulls[i] ? null : (double) floats[i]);
}
} else {
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:
......
/*
* 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.vector.ExprEvalDoubleVector;
import org.apache.druid.math.expr.vector.ExprEvalLongVector;
import org.apache.druid.math.expr.vector.ExprEvalVector;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class ExpressionVectorValueSelectorTest
{
private static final int MAX_SIZE = 8;
private Expr.VectorInputBinding binding;
private ExprVectorProcessor vectorProcessor;
private ExpressionVectorValueSelector expressionVectorValueSelector;
@Before
public void setUp()
{
binding = EasyMock.createMock(Expr.VectorInputBinding.class);
vectorProcessor = EasyMock.createMock(ExprVectorProcessor.class);
EasyMock.expect(binding.getMaxVectorSize()).andReturn(MAX_SIZE).once();
EasyMock.replay(binding, vectorProcessor);
expressionVectorValueSelector = new ExpressionVectorValueSelector(vectorProcessor, binding);
EasyMock.reset(binding, vectorProcessor);
}
@After
public void tearDown()
{
EasyMock.verify(binding, vectorProcessor);
}
@Test
public void testLongVector()
{
final long[] vector = new long[]{1L, 2L, 0L, 3L};
final boolean[] nulls = new boolean[]{false, false, true, false};
ExprEvalVector vectorEval = new ExprEvalLongVector(vector, nulls);
EasyMock.expect(binding.getCurrentVectorId()).andReturn(1).anyTimes();
EasyMock.expect(vectorProcessor.evalVector(binding)).andReturn(vectorEval).once();
EasyMock.replay(binding, vectorProcessor);
long[] vector1 = expressionVectorValueSelector.getLongVector();
boolean[] bools1 = expressionVectorValueSelector.getNullVector();
long[] vector2 = expressionVectorValueSelector.getLongVector();
boolean[] bools2 = expressionVectorValueSelector.getNullVector();
Assert.assertArrayEquals(vector1, vector2);
Assert.assertArrayEquals(bools1, bools2);
}
@Test
public void testDoubleVector()
{
final double[] vector = new double[]{1.0, 2.0, 0.0, 3.0};
final boolean[] nulls = new boolean[]{false, false, true, false};
ExprEvalVector vectorEval = new ExprEvalDoubleVector(vector, nulls);
EasyMock.expect(binding.getCurrentVectorId()).andReturn(1).anyTimes();
EasyMock.expect(vectorProcessor.evalVector(binding)).andReturn(vectorEval).once();
EasyMock.replay(binding, vectorProcessor);
double[] vector1 = expressionVectorValueSelector.getDoubleVector();
boolean[] bools1 = expressionVectorValueSelector.getNullVector();
double[] vector2 = expressionVectorValueSelector.getDoubleVector();
boolean[] bools2 = expressionVectorValueSelector.getNullVector();
Assert.assertArrayEquals(vector1, vector2, 0.0);
Assert.assertArrayEquals(bools1, bools2);
}
@Test
public void testFloatVector()
{
final double[] vector = new double[]{1.0, 2.0, 0.0, 3.0};
final boolean[] nulls = new boolean[]{false, false, true, false};
ExprEvalVector vectorEval = new ExprEvalDoubleVector(vector, nulls);
EasyMock.expect(binding.getCurrentVectorId()).andReturn(1).anyTimes();
EasyMock.expect(binding.getCurrentVectorSize()).andReturn(4).anyTimes();
EasyMock.expect(vectorProcessor.evalVector(binding)).andReturn(vectorEval).once();
EasyMock.replay(binding, vectorProcessor);
float[] vector1 = expressionVectorValueSelector.getFloatVector();
boolean[] bools1 = expressionVectorValueSelector.getNullVector();
float[] vector2 = expressionVectorValueSelector.getFloatVector();
boolean[] bools2 = expressionVectorValueSelector.getNullVector();
for (int i = 0; i < vector1.length; i++) {
Assert.assertEquals(vector1[i], vector2[i], 0.0);
}
Assert.assertArrayEquals(bools1, bools2);
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册