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:
imply-cheddar 2023-02-07 12:36:50 +09:00 committed by GitHub
parent f28c06515b
commit 9c5b61e114
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 1115 additions and 33 deletions

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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

View File

@ -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);
}
}
}

View File

@ -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")

View File

@ -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
)
{

View File

@ -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

View File

@ -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<>();

View File

@ -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(

View File

@ -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
)

View File

@ -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
)
{

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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();
}
}
}

View File

@ -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());
}
}