mirror of https://github.com/apache/druid.git
Fallback virtual column (#13739)
* Fallback virtual column This virtual columns enables falling back to another column if the original column doesn't exist. This is useful when doing column migrations and you have some old data with column X, new data with column Y and you want to use Y if it exists, X otherwise so that you can run a consistent query against all of the data.
This commit is contained in:
parent
f28c06515b
commit
9c5b61e114
|
@ -29,10 +29,10 @@ import org.apache.druid.data.input.MapBasedRow;
|
|||
import org.apache.druid.query.aggregation.AggregatorAdapters;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.groupby.epinephelinae.BufferHashGrouper;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GroupByTestColumnSelectorFactory;
|
||||
import org.apache.druid.query.groupby.epinephelinae.Grouper;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GrouperTestUtil;
|
||||
import org.apache.druid.query.groupby.epinephelinae.IntKey;
|
||||
import org.apache.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -41,7 +41,7 @@ import java.nio.ByteBuffer;
|
|||
public class BufferHashGrouperUsingSketchMergeAggregatorFactoryTest
|
||||
{
|
||||
private static BufferHashGrouper<IntKey> makeGrouper(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
int bufferSize,
|
||||
int initialBuckets
|
||||
)
|
||||
|
@ -68,7 +68,7 @@ public class BufferHashGrouperUsingSketchMergeAggregatorFactoryTest
|
|||
@Test
|
||||
public void testGrowingBufferGrouper()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final Grouper<IntKey> grouper = makeGrouper(columnSelectorFactory, 100000, 2);
|
||||
try {
|
||||
final int expectedMaxSize = 5;
|
||||
|
|
|
@ -46,8 +46,8 @@ import org.apache.druid.query.groupby.GroupByQuery;
|
|||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GroupByTestColumnSelectorFactory;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GrouperTestUtil;
|
||||
import org.apache.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
|
@ -470,6 +470,7 @@ public class SketchAggregationTest
|
|||
public void testSketchAggregatorFactoryComparator()
|
||||
{
|
||||
Comparator<Object> comparator = SketchHolder.COMPARATOR;
|
||||
//noinspection EqualsWithItself
|
||||
Assert.assertEquals(0, comparator.compare(null, null));
|
||||
|
||||
Union union1 = (Union) SetOperation.builder().setNominalEntries(1 << 4).build(Family.UNION);
|
||||
|
@ -497,7 +498,7 @@ public class SketchAggregationTest
|
|||
@Test
|
||||
public void testRelocation()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
SketchHolder sketchHolder = SketchHolder.of(Sketches.updateSketchBuilder().setNominalEntries(16).build());
|
||||
UpdateSketch updateSketch = (UpdateSketch) sketchHolder.getSketch();
|
||||
updateSketch.update(1);
|
||||
|
|
|
@ -38,8 +38,8 @@ import org.apache.druid.query.groupby.GroupByQuery;
|
|||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GroupByTestColumnSelectorFactory;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GrouperTestUtil;
|
||||
import org.apache.druid.query.groupby.epinephelinae.TestColumnSelectorFactory;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -228,7 +228,7 @@ public class OldApiSketchAggregationTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testRelocation()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
SketchHolder sketchHolder = SketchHolder.of(Sketches.updateSketchBuilder().setNominalEntries(16).build());
|
||||
UpdateSketch updateSketch = (UpdateSketch) sketchHolder.getSketch();
|
||||
updateSketch.update(1);
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
|||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.segment.virtual.FallbackVirtualColumn;
|
||||
import org.apache.druid.segment.virtual.ListFilteredVirtualColumn;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -50,6 +51,7 @@ import java.util.List;
|
|||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "expression", value = ExpressionVirtualColumn.class),
|
||||
@JsonSubTypes.Type(name = "fallback", value = FallbackVirtualColumn.class),
|
||||
@JsonSubTypes.Type(name = "mv-filtered", value = ListFilteredVirtualColumn.class)
|
||||
})
|
||||
public interface VirtualColumn extends Cacheable
|
||||
|
|
|
@ -0,0 +1,227 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.virtual;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A virtual column that picks one column or another based on whether they exist. It walks through an array of
|
||||
* DimensionSpecs finding and using the first column that actually exists. If it believes that none of them exist
|
||||
* it YOLOs it with the first entry from the list.
|
||||
* <p>
|
||||
* If you are using this virtual column and want to have a decorator/extraction function on your DimensionSpec,
|
||||
* it is expected that you will put it on the specs in the list rather than on the spec that references this
|
||||
* virtual column. That is, when this virtual column resolves a dimension, it ignores the decoration from the
|
||||
* spec that it was given and instead uses the spec as defined in the list as-is to delegate to the column that
|
||||
* it chose.
|
||||
*/
|
||||
public class FallbackVirtualColumn implements VirtualColumn
|
||||
{
|
||||
private final String name;
|
||||
private final ArrayList<DimensionSpec> columns;
|
||||
|
||||
@JsonCreator
|
||||
public FallbackVirtualColumn(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("columns") ArrayList<DimensionSpec> columns
|
||||
)
|
||||
{
|
||||
if (columns == null || columns.isEmpty()) {
|
||||
throw new IAE("Cannot have a null/empty columns[%s], name[%s]", columns, name);
|
||||
}
|
||||
|
||||
this.name = name;
|
||||
this.columns = columns;
|
||||
}
|
||||
|
||||
@JsonProperty("name")
|
||||
@Override
|
||||
public String getOutputName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty("columns")
|
||||
public ArrayList<DimensionSpec> getColumns()
|
||||
{
|
||||
return columns;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final CacheKeyBuilder bob = new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_LIST_FALLBACK)
|
||||
.appendString(name);
|
||||
|
||||
for (DimensionSpec column : columns) {
|
||||
bob.appendCacheable(column);
|
||||
}
|
||||
|
||||
return bob.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(
|
||||
DimensionSpec dimensionSpec,
|
||||
ColumnSelectorFactory factory
|
||||
)
|
||||
{
|
||||
return factory.makeDimensionSelector(columnToUse(factory));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(
|
||||
String columnName,
|
||||
ColumnSelectorFactory factory
|
||||
)
|
||||
{
|
||||
return factory.makeColumnValueSelector(columnToUse(factory).getDimension());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canVectorize(ColumnInspector inspector)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueVectorDimensionSelector(
|
||||
DimensionSpec dimensionSpec,
|
||||
VectorColumnSelectorFactory factory
|
||||
)
|
||||
{
|
||||
return factory.makeSingleValueDimensionSelector(columnToUse(factory));
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiValueDimensionVectorSelector makeMultiValueVectorDimensionSelector(
|
||||
DimensionSpec dimensionSpec,
|
||||
VectorColumnSelectorFactory factory
|
||||
)
|
||||
{
|
||||
return factory.makeMultiValueDimensionSelector(columnToUse(factory));
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueSelector makeVectorValueSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||
{
|
||||
return factory.makeValueSelector(columnToUse(factory).getDimension());
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||
{
|
||||
return factory.makeObjectSelector(columnToUse(factory).getDimension());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities capabilities(String columnName)
|
||||
{
|
||||
return ColumnCapabilitiesImpl.createDefault();
|
||||
}
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
@Override
|
||||
public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName)
|
||||
{
|
||||
return columnToUseInternal(inspector).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> requiredColumns()
|
||||
{
|
||||
ArrayList<String> retVal = new ArrayList<>(columns.size());
|
||||
for (DimensionSpec column : columns) {
|
||||
retVal.add(column.getDimension());
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean usesDotNotation()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnIndexSupplier getIndexSupplier(String columnName, ColumnSelector columnSelector)
|
||||
{
|
||||
final ColumnHolder columnHolder = columnSelector.getColumnHolder(columnToUse(columnSelector).getDimension());
|
||||
if (columnHolder == null) {
|
||||
return null;
|
||||
}
|
||||
return columnHolder.getIndexSupplier();
|
||||
}
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
private DimensionSpec columnToUse(ColumnInspector inspector)
|
||||
{
|
||||
return columnToUseInternal(inspector).lhs;
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
private Pair<DimensionSpec, ColumnCapabilities> columnToUseInternal(ColumnInspector inspector)
|
||||
{
|
||||
Iterator<DimensionSpec> specs = columns.iterator();
|
||||
|
||||
DimensionSpec firstSpec = specs.next();
|
||||
final ColumnCapabilities firstCapabilities = inspector.getColumnCapabilities(firstSpec.getDimension());
|
||||
|
||||
DimensionSpec spec = firstSpec;
|
||||
ColumnCapabilities capabilities = firstCapabilities;
|
||||
while (capabilities == null && specs.hasNext()) {
|
||||
spec = specs.next();
|
||||
capabilities = inspector.getColumnCapabilities(spec.getDimension());
|
||||
}
|
||||
|
||||
if (capabilities == null) {
|
||||
return Pair.of(firstSpec, firstCapabilities);
|
||||
} else {
|
||||
return Pair.of(spec, capabilities);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ public class VirtualColumnCacheHelper
|
|||
public static final byte CACHE_TYPE_ID_MAP = 0x00;
|
||||
public static final byte CACHE_TYPE_ID_EXPRESSION = 0x01;
|
||||
public static final byte CACHE_TYPE_ID_LIST_FILTERED = 0x02;
|
||||
public static final byte CACHE_TYPE_ID_LIST_FALLBACK = 0x03;
|
||||
|
||||
// Starting byte 0xFF is reserved for site-specific virtual columns.
|
||||
@SuppressWarnings("unused")
|
||||
|
|
|
@ -48,7 +48,7 @@ public class BufferArrayGrouperTest
|
|||
@Test
|
||||
public void testAggregate()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final IntGrouper grouper = newGrouper(columnSelectorFactory, 32768);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -72,7 +72,7 @@ public class BufferArrayGrouperTest
|
|||
}
|
||||
|
||||
private BufferArrayGrouper newGrouper(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
int bufferSize
|
||||
)
|
||||
{
|
||||
|
|
|
@ -52,7 +52,7 @@ public class BufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testSimple()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final Grouper<IntKey> grouper = new BufferHashGrouper<>(
|
||||
Suppliers.ofInstance(ByteBuffer.allocate(1000)),
|
||||
GrouperTestUtil.intKeySerde(),
|
||||
|
@ -99,7 +99,7 @@ public class BufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testGrowing()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
try (final ResourceHolder<Grouper<IntKey>> grouperHolder = makeGrouper(columnSelectorFactory, 10000, 2, 0.75f)) {
|
||||
final Grouper<IntKey> grouper = grouperHolder.get();
|
||||
final int expectedMaxSize = NullHandling.replaceWithDefault() ? 219 : 210;
|
||||
|
@ -132,7 +132,7 @@ public class BufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
// This test checks the bug reported in https://github.com/apache/druid/pull/4333 only when
|
||||
// NullHandling.replaceWithDefault() is true
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
// the buffer size below is chosen to test integer overflow in ByteBufferHashTable.adjustTableWhenFull().
|
||||
try (final ResourceHolder<Grouper<IntKey>> holder = makeGrouper(columnSelectorFactory, 1_900_000_000, 2, 0.3f)) {
|
||||
final Grouper<IntKey> grouper = holder.get();
|
||||
|
@ -150,7 +150,7 @@ public class BufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testNoGrowing()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
try (final ResourceHolder<Grouper<IntKey>> grouperHolder =
|
||||
makeGrouper(columnSelectorFactory, 10000, Integer.MAX_VALUE, 0.75f)) {
|
||||
final Grouper<IntKey> grouper = grouperHolder.get();
|
||||
|
@ -179,7 +179,7 @@ public class BufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
private ResourceHolder<Grouper<IntKey>> makeGrouper(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
int bufferSize,
|
||||
int initialBuckets,
|
||||
float maxLoadFactor
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class TestColumnSelectorFactory implements ColumnSelectorFactory
|
||||
public class GroupByTestColumnSelectorFactory implements ColumnSelectorFactory
|
||||
{
|
||||
private ThreadLocal<Row> row = new ThreadLocal<>();
|
||||
|
|
@ -41,9 +41,9 @@ public class GrouperTestUtil
|
|||
return IntKeySerde.INSTANCE;
|
||||
}
|
||||
|
||||
public static TestColumnSelectorFactory newColumnSelectorFactory()
|
||||
public static GroupByTestColumnSelectorFactory newColumnSelectorFactory()
|
||||
{
|
||||
return new TestColumnSelectorFactory();
|
||||
return new GroupByTestColumnSelectorFactory();
|
||||
}
|
||||
|
||||
public static <T> List<Grouper.Entry<T>> sortedEntries(
|
||||
|
|
|
@ -61,7 +61,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testLimitAndBufferSwapping()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouper(columnSelectorFactory, 20000);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -147,14 +147,14 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
{
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage("LimitedBufferHashGrouper initialized with insufficient buffer capacity");
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
makeGrouper(columnSelectorFactory, 10);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinBufferSize()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouper(columnSelectorFactory, 12120);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -212,7 +212,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
expectedException.expect(IllegalStateException.class);
|
||||
expectedException.expectMessage("attempted to add offset after grouper was iterated");
|
||||
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouper(columnSelectorFactory, 12120);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -229,7 +229,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testIteratorOrderByDim()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouperWithOrderBy(
|
||||
columnSelectorFactory,
|
||||
"value",
|
||||
|
@ -261,7 +261,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testIteratorOrderByDimDesc()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouperWithOrderBy(
|
||||
columnSelectorFactory,
|
||||
"value",
|
||||
|
@ -288,7 +288,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testIteratorOrderByAggs()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouperWithOrderBy(
|
||||
columnSelectorFactory,
|
||||
"valueSum",
|
||||
|
@ -320,7 +320,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testIteratorOrderByAggsDesc()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final LimitedBufferHashGrouper<IntKey> grouper = makeGrouperWithOrderBy(
|
||||
columnSelectorFactory,
|
||||
"valueSum",
|
||||
|
@ -350,7 +350,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
private static LimitedBufferHashGrouper<IntKey> makeGrouper(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
int bufferSize
|
||||
)
|
||||
{
|
||||
|
@ -376,7 +376,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
private static LimitedBufferHashGrouper<IntKey> makeGrouperWithOrderBy(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
String orderByColumn,
|
||||
OrderByColumnSpec.Direction direction
|
||||
)
|
||||
|
|
|
@ -53,7 +53,7 @@ public class StreamingMergeSortedGrouperTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testAggregate()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final StreamingMergeSortedGrouper<IntKey> grouper = newGrouper(columnSelectorFactory, 1024);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -81,7 +81,7 @@ public class StreamingMergeSortedGrouperTest extends InitializedNullHandlingTest
|
|||
@Test(timeout = 60_000L)
|
||||
public void testEmptyIterator()
|
||||
{
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final StreamingMergeSortedGrouper<IntKey> grouper = newGrouper(columnSelectorFactory, 1024);
|
||||
|
||||
grouper.finish();
|
||||
|
@ -104,7 +104,7 @@ public class StreamingMergeSortedGrouperTest extends InitializedNullHandlingTest
|
|||
private void testStreamingAggregate(int bufferSize) throws ExecutionException, InterruptedException
|
||||
{
|
||||
final ExecutorService exec = Execs.multiThreaded(2, "merge-sorted-grouper-test-%d");
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final StreamingMergeSortedGrouper<IntKey> grouper = newGrouper(columnSelectorFactory, bufferSize);
|
||||
|
||||
final List<Entry<IntKey>> expected = new ArrayList<>(1024);
|
||||
|
@ -162,7 +162,7 @@ public class StreamingMergeSortedGrouperTest extends InitializedNullHandlingTest
|
|||
{
|
||||
expectedException.expect(QueryTimeoutException.class);
|
||||
|
||||
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final GroupByTestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
|
||||
final StreamingMergeSortedGrouper<IntKey> grouper = newGrouper(columnSelectorFactory, 100);
|
||||
|
||||
columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L)));
|
||||
|
@ -172,7 +172,7 @@ public class StreamingMergeSortedGrouperTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
private StreamingMergeSortedGrouper<IntKey> newGrouper(
|
||||
TestColumnSelectorFactory columnSelectorFactory,
|
||||
GroupByTestColumnSelectorFactory columnSelectorFactory,
|
||||
int bufferSize
|
||||
)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestColumnSelector implements ColumnSelector
|
||||
{
|
||||
private final Map<String, ColumnHolder> holders = new LinkedHashMap<>();
|
||||
private final Map<String, ColumnCapabilities> capabilitiesMap = new LinkedHashMap<>();
|
||||
|
||||
public TestColumnSelector addHolder(String name, ColumnHolder holder)
|
||||
{
|
||||
holders.put(name, holder);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestColumnSelector addCapabilities(String name, ColumnCapabilities capability)
|
||||
{
|
||||
capabilitiesMap.put(name, capability);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getColumnNames()
|
||||
{
|
||||
return Lists.newArrayList(holders.keySet());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnHolder getColumnHolder(String columnName)
|
||||
{
|
||||
return getFromMap(holders, columnName, "holder");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
return getFromMap(capabilitiesMap, column, "capability");
|
||||
}
|
||||
|
||||
private <T> T getFromMap(Map<String, T> map, String key, String name)
|
||||
{
|
||||
if (!map.containsKey(key)) {
|
||||
throw new UOE("%s[%s] wasn't registered, but was asked for, register first (null is okay)", name, key);
|
||||
}
|
||||
return map.get(key);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestColumnSelectorFactory implements ColumnSelectorFactory
|
||||
{
|
||||
private final Map<String, DimensionSelector> dimSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, ColumnValueSelector<?>> columnSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, ColumnCapabilities> capabilitiesMap = new LinkedHashMap<>();
|
||||
|
||||
public TestColumnSelectorFactory addDimSelector(String name, @Nullable DimensionSelector selector)
|
||||
{
|
||||
dimSelectors.put(name, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public <T> TestColumnSelectorFactory addColumnSelector(String name, @Nullable ColumnValueSelector<T> selector)
|
||||
{
|
||||
columnSelectors.put(name, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestColumnSelectorFactory addCapabilities(String name, @Nullable ColumnCapabilities capabilities)
|
||||
{
|
||||
capabilitiesMap.put(name, capabilities);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
|
||||
{
|
||||
return getFromMap(dimSelectors, dimensionSpec.getDimension(), "dimension");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(String columnName)
|
||||
{
|
||||
return getFromMap(columnSelectors, columnName, "column");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
return getFromMap(capabilitiesMap, column, "capability");
|
||||
}
|
||||
|
||||
private <T> T getFromMap(Map<String, T> map, String key, String name)
|
||||
{
|
||||
if (!map.containsKey(key)) {
|
||||
throw new UOE("%s[%s] wasn't registered, but was asked for, register first (null is okay)", name, key);
|
||||
}
|
||||
return map.get(key);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,114 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestVectorColumnSelectorFactory implements VectorColumnSelectorFactory
|
||||
{
|
||||
private ReadableVectorInspector inspector = null;
|
||||
|
||||
private final Map<String, SingleValueDimensionVectorSelector> singleValDimSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, MultiValueDimensionVectorSelector> multiValDimSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, VectorValueSelector> vectorValueSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, VectorObjectSelector> vectorObjectSelectors = new LinkedHashMap<>();
|
||||
private final Map<String, ColumnCapabilities> capabilitiesMap = new LinkedHashMap<>();
|
||||
|
||||
public TestVectorColumnSelectorFactory addSVDVS(String col, SingleValueDimensionVectorSelector selector)
|
||||
{
|
||||
singleValDimSelectors.put(col, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestVectorColumnSelectorFactory addMVDVS(String col, MultiValueDimensionVectorSelector selector)
|
||||
{
|
||||
multiValDimSelectors.put(col, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestVectorColumnSelectorFactory addVVS(String col, VectorValueSelector selector)
|
||||
{
|
||||
vectorValueSelectors.put(col, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestVectorColumnSelectorFactory addVOS(String col, VectorObjectSelector selector)
|
||||
{
|
||||
vectorObjectSelectors.put(col, selector);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TestVectorColumnSelectorFactory addCapabilities(String col, ColumnCapabilities capabilities)
|
||||
{
|
||||
capabilitiesMap.put(col, capabilities);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableVectorInspector getReadableVectorInspector()
|
||||
{
|
||||
return inspector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionSelector(DimensionSpec dimensionSpec)
|
||||
{
|
||||
return getFromMap(singleValDimSelectors, dimensionSpec.getDimension(), "dimension");
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(DimensionSpec dimensionSpec)
|
||||
{
|
||||
return getFromMap(multiValDimSelectors, dimensionSpec.getDimension(), "dimension");
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorValueSelector makeValueSelector(String column)
|
||||
{
|
||||
return getFromMap(vectorValueSelectors, column, "column");
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeObjectSelector(String column)
|
||||
{
|
||||
return getFromMap(vectorObjectSelectors, column, "column");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnCapabilities getColumnCapabilities(String column)
|
||||
{
|
||||
return getFromMap(capabilitiesMap, column, "capability");
|
||||
}
|
||||
|
||||
private <T> T getFromMap(Map<String, T> map, String key, String name)
|
||||
{
|
||||
if (!map.containsKey(key)) {
|
||||
throw new UOE("%s[%s] wasn't registered, but was asked for, register first (null is okay)", name, key);
|
||||
}
|
||||
return map.get(key);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,581 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.virtual;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.segment.ConstantDimensionSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.TestColumnSelector;
|
||||
import org.apache.druid.segment.TestColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.BaseColumn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.selector.settable.SettableColumnValueSelector;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
||||
@SuppressWarnings("ALL")
|
||||
public class FallbackVirtualColumnTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOutputName()
|
||||
{
|
||||
Assert.assertEquals("slimshady", makeCol("slimshady", "test1", "test2").getOutputName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetColumns()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
Arrays.asList(DefaultDimensionSpec.of("test1"), DefaultDimensionSpec.of("test2")),
|
||||
makeCol("slimshady", "test1", "test2").getColumns()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCacheKey()
|
||||
{
|
||||
Assert.assertArrayEquals(
|
||||
new CacheKeyBuilder((byte) 0x3)
|
||||
.appendString("slimshady")
|
||||
.appendCacheable(DefaultDimensionSpec.of("test1"))
|
||||
.appendCacheable(DefaultDimensionSpec.of("test2"))
|
||||
.appendCacheable(DefaultDimensionSpec.of("oneMore"))
|
||||
.build(),
|
||||
makeCol("slimshady", "test1", "test2", "oneMore").getCacheKey()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeDimensionSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final ConstantDimensionSelector colA = new ConstantDimensionSelector("colA");
|
||||
final ConstantDimensionSelector colB = new ConstantDimensionSelector("colB");
|
||||
final ConstantDimensionSelector colC = new ConstantDimensionSelector("colC");
|
||||
final TestColumnSelectorFactory selectorFactory = new TestColumnSelectorFactory()
|
||||
.addDimSelector("colA", colA)
|
||||
.addDimSelector("colB", colB)
|
||||
.addDimSelector("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeColumnValueSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final ConstantDimensionSelector colA = new ConstantDimensionSelector("colA");
|
||||
final ConstantDimensionSelector colB = new ConstantDimensionSelector("colB");
|
||||
final ConstantDimensionSelector colC = new ConstantDimensionSelector("colC");
|
||||
final TestColumnSelectorFactory selectorFactory = new TestColumnSelectorFactory()
|
||||
.addColumnSelector("colA", colA)
|
||||
.addColumnSelector("colB", colB)
|
||||
.addColumnSelector("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeColumnValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeColumnValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeColumnValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeColumnValueSelector("abcd", selectorFactory));
|
||||
}
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
@Test
|
||||
public void testCanVectorize()
|
||||
{
|
||||
Assert.assertTrue(makeCol("slimshady", "test1").canVectorize(null));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeSingleValueVectorDimensionSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final SameVectorSelector colA = new SameVectorSelector();
|
||||
final SameVectorSelector colB = new SameVectorSelector();
|
||||
final SameVectorSelector colC = new SameVectorSelector();
|
||||
final TestVectorColumnSelectorFactory selectorFactory = new TestVectorColumnSelectorFactory()
|
||||
.addSVDVS("colA", colA)
|
||||
.addSVDVS("colB", colB)
|
||||
.addSVDVS("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeSingleValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeSingleValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeSingleValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeSingleValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeMultiValueVectorDimensionSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final SameMultiVectorSelector colA = new SameMultiVectorSelector();
|
||||
final SameMultiVectorSelector colB = new SameMultiVectorSelector();
|
||||
final SameMultiVectorSelector colC = new SameMultiVectorSelector();
|
||||
final TestVectorColumnSelectorFactory selectorFactory = new TestVectorColumnSelectorFactory()
|
||||
.addMVDVS("colA", colA)
|
||||
.addMVDVS("colB", colB)
|
||||
.addMVDVS("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeMultiValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeMultiValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeMultiValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeMultiValueVectorDimensionSelector(new IgnoredDimensionSpec(), selectorFactory));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeVectorValueSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final SameVectorSelector colA = new SameVectorSelector();
|
||||
final SameVectorSelector colB = new SameVectorSelector();
|
||||
final SameVectorSelector colC = new SameVectorSelector();
|
||||
final TestVectorColumnSelectorFactory selectorFactory = new TestVectorColumnSelectorFactory()
|
||||
.addVVS("colA", colA)
|
||||
.addVVS("colB", colB)
|
||||
.addVVS("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeVectorValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeVectorValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeVectorValueSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeVectorValueSelector("abcd", selectorFactory));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeVectorObjectSelector()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final SameVectorSelector colA = new SameVectorSelector();
|
||||
final SameVectorSelector colB = new SameVectorSelector();
|
||||
final SameVectorSelector colC = new SameVectorSelector();
|
||||
final TestVectorColumnSelectorFactory selectorFactory = new TestVectorColumnSelectorFactory()
|
||||
.addVOS("colA", colA)
|
||||
.addVOS("colB", colB)
|
||||
.addVOS("colC", colC)
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.makeVectorObjectSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.makeVectorObjectSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.makeVectorObjectSelector("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.makeVectorObjectSelector("abcd", selectorFactory));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCapabilities()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final ColumnCapabilitiesImpl colA = ColumnCapabilitiesImpl.createDefault();
|
||||
final ColumnCapabilitiesImpl colB = ColumnCapabilitiesImpl.createDefault();
|
||||
final ColumnCapabilitiesImpl colC = ColumnCapabilitiesImpl.createDefault();
|
||||
final TestVectorColumnSelectorFactory selectorFactory = new TestVectorColumnSelectorFactory()
|
||||
.addCapabilities("colA", colA)
|
||||
.addCapabilities("colB", colB)
|
||||
.addCapabilities("colC", colC);
|
||||
|
||||
Assert.assertEquals(ColumnCapabilitiesImpl.createDefault().getType(), col.capabilities("abcd").getType());
|
||||
|
||||
Assert.assertSame(colA, col.capabilities(selectorFactory, "abcd"));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.capabilities(selectorFactory, "abcd"));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.capabilities(selectorFactory, "abcd"));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertNull(col.capabilities(selectorFactory, "abcd"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequiredColumns()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
Arrays.asList("colA", "colB", "oneMore"),
|
||||
makeCol("slimshady", "colA", "colB", "oneMore").requiredColumns()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUsesDotNotation()
|
||||
{
|
||||
Assert.assertFalse(makeCol("hi", "my", "name", "is").usesDotNotation());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetIndexSupplier()
|
||||
{
|
||||
final FallbackVirtualColumn col = makeCol("slimshady", "colA", "colB", "colC");
|
||||
|
||||
final SameColumnIndexSupplier colA = new SameColumnIndexSupplier();
|
||||
final SameColumnIndexSupplier colB = new SameColumnIndexSupplier();
|
||||
final SameColumnIndexSupplier colC = new SameColumnIndexSupplier();
|
||||
final TestColumnSelector selectorFactory = new TestColumnSelector()
|
||||
.addHolder("colA", new HolderForIndexSupplier(colA))
|
||||
.addHolder("colB", new HolderForIndexSupplier(colB))
|
||||
.addHolder("colC", new HolderForIndexSupplier(colC))
|
||||
.addCapabilities("colA", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colB", ColumnCapabilitiesImpl.createDefault())
|
||||
.addCapabilities("colC", ColumnCapabilitiesImpl.createDefault());
|
||||
|
||||
Assert.assertSame(colA, col.getIndexSupplier("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colA", null);
|
||||
Assert.assertSame(colB, col.getIndexSupplier("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colB", null);
|
||||
Assert.assertSame(colC, col.getIndexSupplier("abcd", selectorFactory));
|
||||
|
||||
selectorFactory.addCapabilities("colC", null);
|
||||
Assert.assertSame(colA, col.getIndexSupplier("abcd", selectorFactory));
|
||||
|
||||
}
|
||||
|
||||
private static FallbackVirtualColumn makeCol(String name, String... cols)
|
||||
{
|
||||
return makeCol(name, Arrays.stream(cols).map(DefaultDimensionSpec::of).toArray(DimensionSpec[]::new));
|
||||
}
|
||||
|
||||
private static FallbackVirtualColumn makeCol(String name, DimensionSpec... specs)
|
||||
{
|
||||
return new FallbackVirtualColumn(name, new ArrayList<>(Arrays.asList(specs)));
|
||||
}
|
||||
|
||||
private static class IgnoredDimensionSpec implements DimensionSpec
|
||||
{
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDimension()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getOutputName()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getOutputType()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ExtractionFn getExtractionFn()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector decorate(DimensionSelector selector)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean mustDecorate()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSpec withDimension(String newDimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private static class SameVectorSelector implements SingleValueDimensionVectorSelector, VectorValueSelector,
|
||||
VectorObjectSelector
|
||||
{
|
||||
@Override
|
||||
public int[] getRowVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long[] getLongVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float[] getFloatVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getDoubleVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private static class SameColumnIndexSupplier implements ColumnIndexSupplier
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private static class SameMultiVectorSelector implements MultiValueDimensionVectorSelector
|
||||
{
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts[] getRowVector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private static class HolderForIndexSupplier implements ColumnHolder
|
||||
{
|
||||
private final ColumnIndexSupplier indexSupplier;
|
||||
|
||||
public HolderForIndexSupplier(
|
||||
ColumnIndexSupplier indexSupplier
|
||||
)
|
||||
{
|
||||
this.indexSupplier = indexSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BaseColumn getColumn()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ColumnIndexSupplier getIndexSupplier()
|
||||
{
|
||||
return indexSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SettableColumnValueSelector<?> makeNewSettableColumnValueSelector()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -48,7 +48,6 @@ public class LocalLoadSpec implements LoadSpec
|
|||
{
|
||||
Preconditions.checkNotNull(path);
|
||||
this.path = Paths.get(path);
|
||||
Preconditions.checkArgument(Files.exists(Paths.get(path)), "[%s] does not exist", path);
|
||||
this.puller = puller;
|
||||
}
|
||||
|
||||
|
@ -61,6 +60,7 @@ public class LocalLoadSpec implements LoadSpec
|
|||
@Override
|
||||
public LoadSpecResult loadSegment(final File outDir) throws SegmentLoadingException
|
||||
{
|
||||
Preconditions.checkArgument(Files.exists(path), "[%s] does not exist", path);
|
||||
return new LoadSpecResult(puller.getSegmentFiles(path.toFile(), outDir).size());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue