mirror of https://github.com/apache/druid.git
* * Add DimensionSelector.idLookup() and nameLookupPossibleInAdvance() to allow better inspection of features DimensionSelectors supports, and safer code working with DimensionSelectors in BaseTopNAlgorithm, BaseFilteredDimensionSpec, DimensionSelectorUtils; * Add PredicateFilteringDimensionSelector, to make BaseFilteredDimensionSpec to be able to decorate DimensionSelectors with unknown cardinality; * Add DimensionSelector.makeValueMatcher() (two kinds) for DimensionSelector-side specifics-aware optimization of ValueMatchers; * Optimize getRow() in BaseFilteredDimensionSpec's DimensionSelector, StringDimensionIndexer's DimensionSelector and SingleScanTimeDimSelector; * Use two static singletons, TrueValueMatcher and FalseValueMatcher, instead of BooleanValueMatcher; * Add NullStringObjectColumnSelector singleton and use it in MapVirtualColumn * Rename DimensionSelectorUtils.makeNonDictionaryEncodedIndexedIntsBasedValueMatcher to makeNonDictionaryEncodedRowBasedValueMatcher * Make ArrayBasedIndexedInts constructor private, replace it's usages with of() static factory method * Cache baseIdLookup in ForwardingFilteredDimensionSelector * Fix a bug in DimensionSelectorUtils.makeRowBasedValueMatcher(selector, predicate, matchNull) * Employ precomputed BitSet optimization in DimensionSelector.makeValueMatcher(value, matchNull) when lookupId() is not available, but cardinality is known and lookupName() is available * Doc fixes * Addressed comments * Fix * Fix * Adjust javadoc of DimensionSelector.nameLookupPossibleInAdvance() for SingleScanTimeDimSelector * throw UnsupportedOperationException instead of IAE in BaseTopNAlgorithm
This commit is contained in:
parent
3136dfa421
commit
75d9e5e7a7
|
@ -30,6 +30,7 @@ import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -94,33 +95,65 @@ public class MapVirtualColumn implements VirtualColumn
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
final int keyId = keySelector.lookupId(dimension.substring(index + 1));
|
IdLookup keyIdLookup = keySelector.idLookup();
|
||||||
|
if (keyIdLookup != null) {
|
||||||
return new ObjectColumnSelector<String>()
|
final int keyId = keyIdLookup.lookupId(dimension.substring(index + 1));
|
||||||
{
|
if (keyId < 0) {
|
||||||
@Override
|
return NullStringObjectColumnSelector.instance();
|
||||||
public Class classOfObject()
|
|
||||||
{
|
|
||||||
return String.class;
|
|
||||||
}
|
}
|
||||||
|
return new ObjectColumnSelector<String>()
|
||||||
@Override
|
|
||||||
public String get()
|
|
||||||
{
|
{
|
||||||
final IndexedInts keyIndices = keySelector.getRow();
|
@Override
|
||||||
final IndexedInts valueIndices = valueSelector.getRow();
|
public Class classOfObject()
|
||||||
if (keyIndices == null || valueIndices == null) {
|
{
|
||||||
|
return String.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String get()
|
||||||
|
{
|
||||||
|
final IndexedInts keyIndices = keySelector.getRow();
|
||||||
|
final IndexedInts valueIndices = valueSelector.getRow();
|
||||||
|
if (keyIndices == null || valueIndices == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final int limit = Math.min(keyIndices.size(), valueIndices.size());
|
||||||
|
for (int i = 0; i < limit; i++) {
|
||||||
|
if (keyIndices.get(i) == keyId) {
|
||||||
|
return valueSelector.lookupName(valueIndices.get(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
final int limit = Math.min(keyIndices.size(), valueIndices.size());
|
};
|
||||||
for (int i = 0; i < limit; i++) {
|
} else {
|
||||||
if (keyIndices.get(i) == keyId) {
|
final String key = dimension.substring(index + 1);
|
||||||
return valueSelector.lookupName(valueIndices.get(i));
|
return new ObjectColumnSelector<String>()
|
||||||
}
|
{
|
||||||
|
@Override
|
||||||
|
public Class classOfObject()
|
||||||
|
{
|
||||||
|
return String.class;
|
||||||
}
|
}
|
||||||
return null;
|
|
||||||
}
|
@Override
|
||||||
};
|
public String get()
|
||||||
|
{
|
||||||
|
final IndexedInts keyIndices = keySelector.getRow();
|
||||||
|
final IndexedInts valueIndices = valueSelector.getRow();
|
||||||
|
if (keyIndices == null || valueIndices == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
final int limit = Math.min(keyIndices.size(), valueIndices.size());
|
||||||
|
for (int i = 0; i < limit; i++) {
|
||||||
|
if (Objects.equals(keySelector.lookupName(keyIndices.get(i)), key)) {
|
||||||
|
return valueSelector.lookupName(valueIndices.get(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -22,13 +22,6 @@ package io.druid.query.dimension;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.DimensionSelector;
|
|
||||||
import io.druid.segment.data.IndexedInts;
|
|
||||||
import io.druid.segment.data.ListBasedIndexedInts;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -72,51 +65,4 @@ public abstract class BaseFilteredDimensionSpec implements DimensionSpec
|
||||||
{
|
{
|
||||||
return delegate.preservesOrdering();
|
return delegate.preservesOrdering();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static DimensionSelector decorate(
|
|
||||||
final DimensionSelector selector,
|
|
||||||
final Map<Integer, Integer> forwardMapping,
|
|
||||||
final int[] reverseMapping
|
|
||||||
)
|
|
||||||
{
|
|
||||||
if (selector == null) {
|
|
||||||
return selector;
|
|
||||||
}
|
|
||||||
|
|
||||||
return new DimensionSelector()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public IndexedInts getRow()
|
|
||||||
{
|
|
||||||
IndexedInts baseRow = selector.getRow();
|
|
||||||
List<Integer> result = new ArrayList<>(baseRow.size());
|
|
||||||
|
|
||||||
for (int i : baseRow) {
|
|
||||||
if (forwardMapping.containsKey(i)) {
|
|
||||||
result.add(forwardMapping.get(i));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return new ListBasedIndexedInts(result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int getValueCardinality()
|
|
||||||
{
|
|
||||||
return forwardMapping.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String lookupName(int id)
|
|
||||||
{
|
|
||||||
return selector.lookupName(reverseMapping[id]);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int lookupId(String name)
|
|
||||||
{
|
|
||||||
return forwardMapping.get(selector.lookupId(name));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,176 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.dimension;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
|
import io.druid.segment.data.ArrayBasedIndexedInts;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
import it.unimi.dsi.fastutil.ints.Int2IntMap;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.BitSet;
|
||||||
|
|
||||||
|
final class ForwardingFilteredDimensionSelector implements DimensionSelector, IdLookup
|
||||||
|
{
|
||||||
|
private final DimensionSelector selector;
|
||||||
|
private final IdLookup baseIdLookup;
|
||||||
|
private final Int2IntMap forwardMapping;
|
||||||
|
private final int[] reverseMapping;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param selector must return true from {@link DimensionSelector#nameLookupPossibleInAdvance()}
|
||||||
|
* @param forwardMapping must have {@link Int2IntMap#defaultReturnValue(int)} configured to -1.
|
||||||
|
*/
|
||||||
|
ForwardingFilteredDimensionSelector(DimensionSelector selector, Int2IntMap forwardMapping, int[] reverseMapping)
|
||||||
|
{
|
||||||
|
this.selector = Preconditions.checkNotNull(selector);
|
||||||
|
if (!selector.nameLookupPossibleInAdvance()) {
|
||||||
|
throw new IAE("selector.nameLookupPossibleInAdvance() should return true");
|
||||||
|
}
|
||||||
|
this.baseIdLookup = selector.idLookup();
|
||||||
|
this.forwardMapping = Preconditions.checkNotNull(forwardMapping);
|
||||||
|
if (forwardMapping.defaultReturnValue() != -1) {
|
||||||
|
throw new IAE("forwardMapping.defaultReturnValue() should be -1");
|
||||||
|
}
|
||||||
|
this.reverseMapping = Preconditions.checkNotNull(reverseMapping);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts getRow()
|
||||||
|
{
|
||||||
|
IndexedInts baseRow = selector.getRow();
|
||||||
|
int baseRowSize = baseRow.size();
|
||||||
|
int[] result = new int[baseRowSize];
|
||||||
|
int resultSize = 0;
|
||||||
|
for (int i = 0; i < baseRowSize; i++) {
|
||||||
|
int forwardedValue = forwardMapping.get(baseRow.get(i));
|
||||||
|
if (forwardedValue >= 0) {
|
||||||
|
result[resultSize++] = forwardedValue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return ArrayBasedIndexedInts.of(result, resultSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
IdLookup idLookup = idLookup();
|
||||||
|
if (idLookup != null) {
|
||||||
|
final int valueId = idLookup.lookupId(value);
|
||||||
|
if (valueId >= 0 || value == null) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts baseRow = selector.getRow();
|
||||||
|
final int baseRowSize = baseRow.size();
|
||||||
|
boolean nullRow = true;
|
||||||
|
for (int i = 0; i < baseRowSize; i++) {
|
||||||
|
int forwardedValue = forwardMapping.get(baseRow.get(i));
|
||||||
|
if (forwardedValue >= 0) {
|
||||||
|
// Make the following check after the `forwardedValue >= 0` check, because if forwardedValue is -1 and
|
||||||
|
// valueId is -1, we don't want to return true from matches().
|
||||||
|
if (forwardedValue == valueId) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
nullRow = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return nullRow && value == null;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Employ precomputed BitSet optimization
|
||||||
|
return makeValueMatcher(Predicates.equalTo(value));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
final BitSet valueIds = DimensionSelectorUtils.makePredicateMatchingSet(this, predicate);
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts baseRow = selector.getRow();
|
||||||
|
final int baseRowSize = baseRow.size();
|
||||||
|
boolean nullRow = true;
|
||||||
|
for (int i = 0; i < baseRowSize; ++i) {
|
||||||
|
int forwardedValue = forwardMapping.get(baseRow.get(i));
|
||||||
|
if (forwardedValue >= 0) {
|
||||||
|
if (valueIds.get(forwardedValue)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
nullRow = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return nullRow && matchNull;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return forwardMapping.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return selector.lookupName(reverseMapping[id]);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return baseIdLookup != null ? this : null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int lookupId(String name)
|
||||||
|
{
|
||||||
|
return forwardMapping.get(baseIdLookup.lookupId(name));
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,14 +21,18 @@ package io.druid.query.dimension;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.query.filter.DimFilterUtils;
|
import io.druid.query.filter.DimFilterUtils;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
|
import it.unimi.dsi.fastutil.ints.Int2IntMap;
|
||||||
|
import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -71,24 +75,31 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
||||||
public DimensionSelector decorate(final DimensionSelector selector)
|
public DimensionSelector decorate(final DimensionSelector selector)
|
||||||
{
|
{
|
||||||
if (selector == null) {
|
if (selector == null) {
|
||||||
return selector;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final int selectorCardinality = selector.getValueCardinality();
|
|
||||||
if (selectorCardinality < 0) {
|
|
||||||
throw new UnsupportedOperationException("Cannot decorate a selector with no dictionary");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Upper bound on cardinality of the filtered spec.
|
|
||||||
final int cardinality = isWhitelist ? values.size() : selectorCardinality;
|
|
||||||
|
|
||||||
int count = 0;
|
|
||||||
final Map<Integer,Integer> forwardMapping = new HashMap<>(cardinality);
|
|
||||||
final int[] reverseMapping = new int[cardinality];
|
|
||||||
|
|
||||||
if (isWhitelist) {
|
if (isWhitelist) {
|
||||||
|
return filterWhiteList(selector);
|
||||||
|
} else {
|
||||||
|
return filterBlackList(selector);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private DimensionSelector filterWhiteList(DimensionSelector selector)
|
||||||
|
{
|
||||||
|
final int selectorCardinality = selector.getValueCardinality();
|
||||||
|
if (selectorCardinality < 0 || (selector.idLookup() == null && !selector.nameLookupPossibleInAdvance())) {
|
||||||
|
return new PredicateFilteredDimensionSelector(selector, Predicates.in(values));
|
||||||
|
}
|
||||||
|
final int maxPossibleFilteredCardinality = values.size();
|
||||||
|
int count = 0;
|
||||||
|
final Int2IntMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality);
|
||||||
|
forwardMapping.defaultReturnValue(-1);
|
||||||
|
final int[] reverseMapping = new int[maxPossibleFilteredCardinality];
|
||||||
|
IdLookup idLookup = selector.idLookup();
|
||||||
|
if (idLookup != null) {
|
||||||
for (String value : values) {
|
for (String value : values) {
|
||||||
int i = selector.lookupId(value);
|
int i = idLookup.lookupId(value);
|
||||||
if (i >= 0) {
|
if (i >= 0) {
|
||||||
forwardMapping.put(i, count);
|
forwardMapping.put(i, count);
|
||||||
reverseMapping[count++] = i;
|
reverseMapping[count++] = i;
|
||||||
|
@ -96,14 +107,43 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
for (int i = 0; i < selectorCardinality; i++) {
|
for (int i = 0; i < selectorCardinality; i++) {
|
||||||
if (!values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
if (values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
||||||
forwardMapping.put(i, count);
|
forwardMapping.put(i, count);
|
||||||
reverseMapping[count++] = i;
|
reverseMapping[count++] = i;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return new ForwardingFilteredDimensionSelector(selector, forwardMapping, reverseMapping);
|
||||||
|
}
|
||||||
|
|
||||||
return BaseFilteredDimensionSpec.decorate(selector, forwardMapping, reverseMapping);
|
private DimensionSelector filterBlackList(DimensionSelector selector)
|
||||||
|
{
|
||||||
|
final int selectorCardinality = selector.getValueCardinality();
|
||||||
|
if (selectorCardinality < 0 || !selector.nameLookupPossibleInAdvance()) {
|
||||||
|
return new PredicateFilteredDimensionSelector(
|
||||||
|
selector,
|
||||||
|
new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable String input)
|
||||||
|
{
|
||||||
|
return !values.contains(input);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
final int maxPossibleFilteredCardinality = selectorCardinality;
|
||||||
|
int count = 0;
|
||||||
|
final Int2IntMap forwardMapping = new Int2IntOpenHashMap(maxPossibleFilteredCardinality);
|
||||||
|
forwardMapping.defaultReturnValue(-1);
|
||||||
|
final int[] reverseMapping = new int[maxPossibleFilteredCardinality];
|
||||||
|
for (int i = 0; i < selectorCardinality; i++) {
|
||||||
|
if (!values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
||||||
|
forwardMapping.put(i, count);
|
||||||
|
reverseMapping[count++] = i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new ForwardingFilteredDimensionSelector(selector, forwardMapping, reverseMapping);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,135 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.dimension;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
|
import io.druid.segment.data.ArrayBasedIndexedInts;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
final class PredicateFilteredDimensionSelector implements DimensionSelector
|
||||||
|
{
|
||||||
|
private final DimensionSelector selector;
|
||||||
|
private final Predicate<String> predicate;
|
||||||
|
|
||||||
|
PredicateFilteredDimensionSelector(DimensionSelector selector, Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
this.selector = selector;
|
||||||
|
this.predicate = predicate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts getRow()
|
||||||
|
{
|
||||||
|
IndexedInts baseRow = selector.getRow();
|
||||||
|
int baseRowSize = baseRow.size();
|
||||||
|
int[] result = new int[baseRowSize];
|
||||||
|
int resultSize = 0;
|
||||||
|
for (int i = 0; i < baseRowSize; i++) {
|
||||||
|
if (predicate.apply(selector.lookupName(baseRow.get(i)))) {
|
||||||
|
result[resultSize++] = i;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return ArrayBasedIndexedInts.of(result, resultSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts baseRow = selector.getRow();
|
||||||
|
final int baseRowSize = baseRow.size();
|
||||||
|
boolean nullRow = true;
|
||||||
|
for (int i = 0; i < baseRowSize; i++) {
|
||||||
|
String rowValue = lookupName(baseRow.get(i));
|
||||||
|
if (predicate.apply(rowValue)) {
|
||||||
|
if (Objects.equals(rowValue, value)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
nullRow = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return nullRow && value == null;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> matcherPredicate)
|
||||||
|
{
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts baseRow = selector.getRow();
|
||||||
|
final int baseRowSize = baseRow.size();
|
||||||
|
boolean nullRow = true;
|
||||||
|
for (int i = 0; i < baseRowSize; ++i) {
|
||||||
|
String rowValue = lookupName(baseRow.get(i));
|
||||||
|
if (predicate.apply(rowValue)) {
|
||||||
|
if (matcherPredicate.apply(rowValue)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
nullRow = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return nullRow && matchNull;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return selector.getValueCardinality();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return selector.lookupName(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return selector.nameLookupPossibleInAdvance();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return selector.idLookup();
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,14 +21,16 @@ package io.druid.query.dimension;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.query.filter.DimFilterUtils;
|
import io.druid.query.filter.DimFilterUtils;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import it.unimi.dsi.fastutil.ints.Int2IntMap;
|
||||||
|
import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -62,17 +64,27 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
||||||
public DimensionSelector decorate(final DimensionSelector selector)
|
public DimensionSelector decorate(final DimensionSelector selector)
|
||||||
{
|
{
|
||||||
if (selector == null) {
|
if (selector == null) {
|
||||||
return selector;
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final int selectorCardinality = selector.getValueCardinality();
|
||||||
|
if (selectorCardinality < 0 || !selector.nameLookupPossibleInAdvance()) {
|
||||||
|
return new PredicateFilteredDimensionSelector(
|
||||||
|
selector,
|
||||||
|
new Predicate<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean apply(@Nullable String input)
|
||||||
|
{
|
||||||
|
return compiledRegex.matcher(Strings.nullToEmpty(input)).matches();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
int count = 0;
|
int count = 0;
|
||||||
final Map<Integer,Integer> forwardMapping = new HashMap<>();
|
final Int2IntMap forwardMapping = new Int2IntOpenHashMap();
|
||||||
|
forwardMapping.defaultReturnValue(-1);
|
||||||
final int selectorCardinality = selector.getValueCardinality();
|
|
||||||
if (selectorCardinality < 0) {
|
|
||||||
throw new UnsupportedOperationException("Cannot decorate a selector with no dictionary");
|
|
||||||
}
|
|
||||||
|
|
||||||
for (int i = 0; i < selectorCardinality; i++) {
|
for (int i = 0; i < selectorCardinality; i++) {
|
||||||
if (compiledRegex.matcher(Strings.nullToEmpty(selector.lookupName(i))).matches()) {
|
if (compiledRegex.matcher(Strings.nullToEmpty(selector.lookupName(i))).matches()) {
|
||||||
forwardMapping.put(i, count++);
|
forwardMapping.put(i, count++);
|
||||||
|
@ -80,10 +92,10 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
final int[] reverseMapping = new int[forwardMapping.size()];
|
final int[] reverseMapping = new int[forwardMapping.size()];
|
||||||
for (Map.Entry<Integer, Integer> e : forwardMapping.entrySet()) {
|
for (Int2IntMap.Entry e : forwardMapping.int2IntEntrySet()) {
|
||||||
reverseMapping[e.getValue().intValue()] = e.getKey().intValue();
|
reverseMapping[e.getIntValue()] = e.getIntKey();
|
||||||
}
|
}
|
||||||
return BaseFilteredDimensionSpec.decorate(selector, forwardMapping, reverseMapping);
|
return new ForwardingFilteredDimensionSelector(selector, forwardMapping, reverseMapping);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -22,73 +22,18 @@ package io.druid.query.filter;
|
||||||
import com.google.common.base.Predicate;
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.data.IndexedInts;
|
|
||||||
import io.druid.segment.filter.BooleanValueMatcher;
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
|
||||||
import java.util.BitSet;
|
|
||||||
import java.util.Objects;
|
|
||||||
|
|
||||||
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
|
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeValueMatcher(final DimensionSelector selector, final String value)
|
public ValueMatcher makeValueMatcher(final DimensionSelector selector, String value)
|
||||||
{
|
{
|
||||||
final String valueStr = Strings.emptyToNull(value);
|
value = Strings.emptyToNull(value);
|
||||||
|
if (selector.getValueCardinality() == 0) {
|
||||||
// if matching against null, rows with size 0 should also match
|
return BooleanValueMatcher.of(value == null);
|
||||||
final boolean matchNull = Strings.isNullOrEmpty(valueStr);
|
|
||||||
|
|
||||||
final int cardinality = selector.getValueCardinality();
|
|
||||||
|
|
||||||
if (cardinality == 0 || (cardinality == 1 && selector.lookupName(0) == null)) {
|
|
||||||
// All values are null or empty rows (which match nulls anyway). No need to check each row.
|
|
||||||
return new BooleanValueMatcher(matchNull);
|
|
||||||
} else if (cardinality >= 0) {
|
|
||||||
// Dictionary-encoded dimension. Compare by id instead of by value to save time.
|
|
||||||
final int valueId = selector.lookupId(valueStr);
|
|
||||||
|
|
||||||
return new ValueMatcher()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public boolean matches()
|
|
||||||
{
|
|
||||||
final IndexedInts row = selector.getRow();
|
|
||||||
final int size = row.size();
|
|
||||||
if (size == 0) {
|
|
||||||
// null should match empty rows in multi-value columns
|
|
||||||
return matchNull;
|
|
||||||
} else {
|
|
||||||
for (int i = 0; i < size; ++i) {
|
|
||||||
if (row.get(i) == valueId) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
} else {
|
} else {
|
||||||
// Not dictionary-encoded. Skip the optimization.
|
return selector.makeValueMatcher(value);
|
||||||
return new ValueMatcher()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public boolean matches()
|
|
||||||
{
|
|
||||||
final IndexedInts row = selector.getRow();
|
|
||||||
final int size = row.size();
|
|
||||||
if (size == 0) {
|
|
||||||
// null should match empty rows in multi-value columns
|
|
||||||
return matchNull;
|
|
||||||
} else {
|
|
||||||
for (int i = 0; i < size; ++i) {
|
|
||||||
if (Objects.equals(selector.lookupName(row.get(i)), valueStr)) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -99,63 +44,11 @@ public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherCol
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||||
final int cardinality = selector.getValueCardinality();
|
if (selector.getValueCardinality() == 0) {
|
||||||
final boolean matchNull = predicate.apply(null);
|
return BooleanValueMatcher.of(predicate.apply(null));
|
||||||
|
|
||||||
if (cardinality == 0 || (cardinality == 1 && selector.lookupName(0) == null)) {
|
|
||||||
// All values are null or empty rows (which match nulls anyway). No need to check each row.
|
|
||||||
return new BooleanValueMatcher(matchNull);
|
|
||||||
} else if (cardinality >= 0) {
|
|
||||||
// Dictionary-encoded dimension. Check every value; build a bitset of matching ids.
|
|
||||||
final BitSet valueIds = new BitSet(cardinality);
|
|
||||||
for (int i = 0; i < cardinality; i++) {
|
|
||||||
if (predicate.apply(selector.lookupName(i))) {
|
|
||||||
valueIds.set(i);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return new ValueMatcher()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public boolean matches()
|
|
||||||
{
|
|
||||||
final IndexedInts row = selector.getRow();
|
|
||||||
final int size = row.size();
|
|
||||||
if (size == 0) {
|
|
||||||
// null should match empty rows in multi-value columns
|
|
||||||
return matchNull;
|
|
||||||
} else {
|
|
||||||
for (int i = 0; i < size; ++i) {
|
|
||||||
if (valueIds.get(row.get(i))) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
} else {
|
} else {
|
||||||
// Not dictionary-encoded. Skip the optimization.
|
return selector.makeValueMatcher(predicate);
|
||||||
return new ValueMatcher()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public boolean matches()
|
|
||||||
{
|
|
||||||
final IndexedInts row = selector.getRow();
|
|
||||||
final int size = row.size();
|
|
||||||
if (size == 0) {
|
|
||||||
// null should match empty rows in multi-value columns
|
|
||||||
return matchNull;
|
|
||||||
} else {
|
|
||||||
for (int i = 0; i < size; ++i) {
|
|
||||||
if (predicate.apply(selector.lookupName(row.get(i)))) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,15 +19,18 @@
|
||||||
|
|
||||||
package io.druid.query.groupby;
|
package io.druid.query.groupby;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
|
@ -41,6 +44,7 @@ import io.druid.segment.data.ZeroIndexedInts;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
{
|
{
|
||||||
|
@ -108,6 +112,34 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
return ZeroIndexedInts.instance();
|
return ZeroIndexedInts.instance();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
String rowValue = extractionFn.apply(row.get().getTimestampFromEpoch());
|
||||||
|
return Objects.equals(rowValue, value);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
String rowValue = extractionFn.apply(row.get().getTimestampFromEpoch());
|
||||||
|
return predicate.apply(rowValue);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -121,9 +153,16 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("lookupId");
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
} else {
|
} else {
|
||||||
|
@ -136,6 +175,95 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
return RangeIndexedInts.create(dimensionValues != null ? dimensionValues.size() : 0);
|
return RangeIndexedInts.create(dimensionValues != null ? dimensionValues.size() : 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
if (extractionFn == null) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final List<String> dimensionValues = row.get().getDimension(dimension);
|
||||||
|
if (dimensionValues == null || dimensionValues.isEmpty()) {
|
||||||
|
return value == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String dimensionValue : dimensionValues) {
|
||||||
|
if (Objects.equals(Strings.emptyToNull(dimensionValue), value)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final List<String> dimensionValues = row.get().getDimension(dimension);
|
||||||
|
if (dimensionValues == null || dimensionValues.isEmpty()) {
|
||||||
|
return value == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String dimensionValue : dimensionValues) {
|
||||||
|
if (Objects.equals(extractionFn.apply(Strings.emptyToNull(dimensionValue)), value)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
if (extractionFn == null) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final List<String> dimensionValues = row.get().getDimension(dimension);
|
||||||
|
if (dimensionValues == null || dimensionValues.isEmpty()) {
|
||||||
|
return matchNull;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String dimensionValue : dimensionValues) {
|
||||||
|
if (predicate.apply(Strings.emptyToNull(dimensionValue))) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final List<String> dimensionValues = row.get().getDimension(dimension);
|
||||||
|
if (dimensionValues == null || dimensionValues.isEmpty()) {
|
||||||
|
return matchNull;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (String dimensionValue : dimensionValues) {
|
||||||
|
if (predicate.apply(extractionFn.apply(Strings.emptyToNull(dimensionValue)))) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -150,9 +278,16 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("lookupId");
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
|
@ -97,7 +97,7 @@ public class GroupByRowProcessor
|
||||||
rowSignature
|
rowSignature
|
||||||
);
|
);
|
||||||
final ValueMatcher filterMatcher = filter == null
|
final ValueMatcher filterMatcher = filter == null
|
||||||
? new BooleanValueMatcher(true)
|
? BooleanValueMatcher.of(true)
|
||||||
: filter.makeMatcher(columnSelectorFactory);
|
: filter.makeMatcher(columnSelectorFactory);
|
||||||
|
|
||||||
final FilteredSequence<Row> filteredSequence = new FilteredSequence<>(
|
final FilteredSequence<Row> filteredSequence = new FilteredSequence<>(
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.query.topn;
|
package io.druid.query.topn;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
import io.druid.java.util.common.Pair;
|
import io.druid.java.util.common.Pair;
|
||||||
import io.druid.query.aggregation.Aggregator;
|
import io.druid.query.aggregation.Aggregator;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
@ -26,6 +27,7 @@ import io.druid.query.aggregation.BufferAggregator;
|
||||||
import io.druid.segment.Capabilities;
|
import io.druid.segment.Capabilities;
|
||||||
import io.druid.segment.Cursor;
|
import io.druid.segment.Cursor;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -179,6 +181,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
private volatile int keepOnlyN;
|
private volatile int keepOnlyN;
|
||||||
|
|
||||||
private final DimensionSelector dimSelector;
|
private final DimensionSelector dimSelector;
|
||||||
|
private final IdLookup idLookup;
|
||||||
private final TopNQuery query;
|
private final TopNQuery query;
|
||||||
private final Capabilities capabilities;
|
private final Capabilities capabilities;
|
||||||
|
|
||||||
|
@ -189,6 +192,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimSelector = dimSelector;
|
this.dimSelector = dimSelector;
|
||||||
|
this.idLookup = dimSelector.idLookup();
|
||||||
this.query = query;
|
this.query = query;
|
||||||
this.capabilities = capabilities;
|
this.capabilities = capabilities;
|
||||||
|
|
||||||
|
@ -198,7 +202,7 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
keepOnlyN = dimSelector.getValueCardinality();
|
keepOnlyN = dimSelector.getValueCardinality();
|
||||||
|
|
||||||
if (keepOnlyN < 0) {
|
if (keepOnlyN < 0) {
|
||||||
throw new UnsupportedOperationException("Cannot operate on a dimension with no dictionary");
|
throw new IAE("Cannot operate on a dimension with no dictionary");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -233,7 +237,10 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
|
||||||
int startIndex = ignoreFirstN;
|
int startIndex = ignoreFirstN;
|
||||||
|
|
||||||
if (previousStop != null) {
|
if (previousStop != null) {
|
||||||
int lookupId = dimSelector.lookupId(previousStop) + 1;
|
if (idLookup == null) {
|
||||||
|
throw new UnsupportedOperationException("Only DimensionSelectors which support idLookup() are supported yet");
|
||||||
|
}
|
||||||
|
int lookupId = idLookup.lookupId(previousStop) + 1;
|
||||||
if (lookupId < 0) {
|
if (lookupId < 0) {
|
||||||
lookupId *= -1;
|
lookupId *= -1;
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,7 +17,13 @@
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package io.druid.segment;import io.druid.segment.data.IndexedInts;
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -34,6 +40,13 @@ public interface DimensionSelector extends ColumnValueSelector
|
||||||
*/
|
*/
|
||||||
public IndexedInts getRow();
|
public IndexedInts getRow();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param value nullable dimension value
|
||||||
|
*/
|
||||||
|
ValueMatcher makeValueMatcher(String value);
|
||||||
|
|
||||||
|
ValueMatcher makeValueMatcher(Predicate<String> predicate);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Value cardinality is the cardinality of the different occurring values. If there were 4 rows:
|
* Value cardinality is the cardinality of the different occurring values. If there were 4 rows:
|
||||||
*
|
*
|
||||||
|
@ -80,10 +93,24 @@ public interface DimensionSelector extends ColumnValueSelector
|
||||||
public String lookupName(int id);
|
public String lookupName(int id);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The ID is the int id value of the field.
|
* Returns true if it is possible to {@link #lookupName(int)} by ids from 0 to {@link #getValueCardinality()}
|
||||||
|
* before the rows with those ids are returned.
|
||||||
*
|
*
|
||||||
* @param name field name to look up the id for
|
* <p>Returns false if {@link #lookupName(int)} could be called with ids, returned from the most recent call of {@link
|
||||||
* @return the id for the given field name
|
* #getRow()} on this DimensionSelector, but not earlier. If {@link #getValueCardinality()} of this DimensionSelector
|
||||||
|
* additionally returns {@link #CARDINALITY_UNKNOWN}, {@code lookupName()} couldn't be called with ids, returned by
|
||||||
|
* not the most recent call of {@link #getRow()}, i. e. names for ids couldn't be looked up "later". If {@link
|
||||||
|
* #getValueCardinality()} returns a non-negative number, {@code lookupName()} could be called with any ids, returned
|
||||||
|
* from {@code #getRow()} since the creation of this DimensionSelector.
|
||||||
|
*
|
||||||
|
* <p>If {@link #lookupName(int)} is called with an ineligible id, result is undefined: exception could be thrown, or
|
||||||
|
* null returned, or some other random value.
|
||||||
*/
|
*/
|
||||||
public int lookupId(String name);
|
boolean nameLookupPossibleInAdvance();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns {@link IdLookup} if available for this DimensionSelector, or null.
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
IdLookup idLookup();
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,217 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
|
||||||
|
import java.util.BitSet;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public final class DimensionSelectorUtils
|
||||||
|
{
|
||||||
|
|
||||||
|
private DimensionSelectorUtils()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generic implementation of {@link DimensionSelector#makeValueMatcher(String)}, uses {@link
|
||||||
|
* DimensionSelector#getRow()} of the given {@link DimensionSelector}. "Lazy" DimensionSelectors could delegate
|
||||||
|
* {@code makeValueMatcher()} to this method, but encouraged to implement {@code makeValueMatcher()} themselves,
|
||||||
|
* bypassing the {@link IndexedInts} abstraction.
|
||||||
|
*/
|
||||||
|
public static ValueMatcher makeValueMatcherGeneric(DimensionSelector selector, String value)
|
||||||
|
{
|
||||||
|
IdLookup idLookup = selector.idLookup();
|
||||||
|
if (idLookup != null) {
|
||||||
|
return makeDictionaryEncodedValueMatcherGeneric(selector, idLookup.lookupId(value), value == null);
|
||||||
|
} else if (selector.getValueCardinality() >= 0 && selector.nameLookupPossibleInAdvance()) {
|
||||||
|
// Employ precomputed BitSet optimization
|
||||||
|
return makeDictionaryEncodedValueMatcherGeneric(selector, Predicates.equalTo(value));
|
||||||
|
} else {
|
||||||
|
return makeNonDictionaryEncodedValueMatcherGeneric(selector, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ValueMatcher makeDictionaryEncodedValueMatcherGeneric(
|
||||||
|
final DimensionSelector selector,
|
||||||
|
final int valueId,
|
||||||
|
final boolean matchNull
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (valueId >= 0) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
if (size == 0) {
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return matchNull;
|
||||||
|
} else {
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
if (row.get(i) == valueId) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
if (matchNull) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
return size == 0;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ValueMatcher makeNonDictionaryEncodedValueMatcherGeneric(
|
||||||
|
final DimensionSelector selector,
|
||||||
|
final String value
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
if (size == 0) {
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return value == null;
|
||||||
|
} else {
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
if (Objects.equals(selector.lookupName(row.get(i)), value)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generic implementation of {@link DimensionSelector#makeValueMatcher(Predicate)}, uses {@link
|
||||||
|
* DimensionSelector#getRow()} of the given {@link DimensionSelector}. "Lazy" DimensionSelectors could delegate
|
||||||
|
* {@code makeValueMatcher()} to this method, but encouraged to implement {@code makeValueMatcher()} themselves,
|
||||||
|
* bypassing the {@link IndexedInts} abstraction.
|
||||||
|
*/
|
||||||
|
public static ValueMatcher makeValueMatcherGeneric(DimensionSelector selector, Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
int cardinality = selector.getValueCardinality();
|
||||||
|
if (cardinality >= 0 && selector.nameLookupPossibleInAdvance()) {
|
||||||
|
return makeDictionaryEncodedValueMatcherGeneric(selector, predicate);
|
||||||
|
} else {
|
||||||
|
return makeNonDictionaryEncodedValueMatcherGeneric(selector, predicate);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ValueMatcher makeDictionaryEncodedValueMatcherGeneric(
|
||||||
|
final DimensionSelector selector,
|
||||||
|
Predicate<String> predicate
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final BitSet predicateMatchingValueIds = makePredicateMatchingSet(selector, predicate);
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
if (size == 0) {
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return matchNull;
|
||||||
|
} else {
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
if (predicateMatchingValueIds.get(row.get(i))) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ValueMatcher makeNonDictionaryEncodedValueMatcherGeneric(
|
||||||
|
final DimensionSelector selector,
|
||||||
|
final Predicate<String> predicate
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
final IndexedInts row = selector.getRow();
|
||||||
|
final int size = row.size();
|
||||||
|
if (size == 0) {
|
||||||
|
// null should match empty rows in multi-value columns
|
||||||
|
return matchNull;
|
||||||
|
} else {
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
if (predicate.apply(selector.lookupName(row.get(i)))) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public static BitSet makePredicateMatchingSet(DimensionSelector selector, Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
if (!selector.nameLookupPossibleInAdvance()) {
|
||||||
|
throw new IAE("selector.nameLookupPossibleInAdvance() should return true");
|
||||||
|
}
|
||||||
|
int cardinality = selector.getValueCardinality();
|
||||||
|
BitSet valueIds = new BitSet(cardinality);
|
||||||
|
for (int i = 0; i < cardinality; i++) {
|
||||||
|
if (predicate.apply(selector.lookupName(i))) {
|
||||||
|
valueIds.set(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return valueIds;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* "Mixin" for {@link DimensionSelector}.
|
||||||
|
*/
|
||||||
|
public interface IdLookup
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Inverse of {@link DimensionSelector#lookupName(int)}.
|
||||||
|
*/
|
||||||
|
int lookupId(String name);
|
||||||
|
}
|
|
@ -19,11 +19,16 @@
|
||||||
|
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.ZeroIndexedInts;
|
import io.druid.segment.data.ZeroIndexedInts;
|
||||||
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
|
|
||||||
public class NullDimensionSelector implements DimensionSelector
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
public class NullDimensionSelector implements DimensionSelector, IdLookup
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
|
@ -31,6 +36,18 @@ public class NullDimensionSelector implements DimensionSelector
|
||||||
return ZeroIndexedInts.instance();
|
return ZeroIndexedInts.instance();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(String value)
|
||||||
|
{
|
||||||
|
return BooleanValueMatcher.of(value == null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return BooleanValueMatcher.of(predicate.apply(null));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -40,9 +57,23 @@ public class NullDimensionSelector implements DimensionSelector
|
||||||
@Override
|
@Override
|
||||||
public String lookupName(int id)
|
public String lookupName(int id)
|
||||||
{
|
{
|
||||||
|
assert id == 0 : "id = " + id;
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public int lookupId(String name)
|
||||||
{
|
{
|
||||||
|
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment;
|
||||||
|
|
||||||
|
public final class NullStringObjectColumnSelector implements ObjectColumnSelector<String>
|
||||||
|
{
|
||||||
|
private static final NullStringObjectColumnSelector INSTANCE = new NullStringObjectColumnSelector();
|
||||||
|
|
||||||
|
public static NullStringObjectColumnSelector instance()
|
||||||
|
{
|
||||||
|
return INSTANCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
private NullStringObjectColumnSelector()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<String> classOfObject()
|
||||||
|
{
|
||||||
|
return String.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String get()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.base.Predicates;
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -59,6 +60,7 @@ import javax.annotation.Nullable;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.BitSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -463,7 +465,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
if (column == null) {
|
if (column == null) {
|
||||||
return NULL_DIMENSION_SELECTOR;
|
return NULL_DIMENSION_SELECTOR;
|
||||||
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
|
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
|
||||||
return new DimensionSelector()
|
class MultiValueDimensionSelector implements DimensionSelector, IdLookup
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
|
@ -471,6 +473,18 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
return column.getMultiValueRow(cursorOffset.getOffset());
|
return column.getMultiValueRow(cursorOffset.getOffset());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(String value)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -486,6 +500,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
extractionFn.apply(value);
|
extractionFn.apply(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return extractionFn == null ? this : null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public int lookupId(String name)
|
||||||
{
|
{
|
||||||
|
@ -496,9 +523,10 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
return column.lookupId(name);
|
return column.lookupId(name);
|
||||||
}
|
}
|
||||||
};
|
}
|
||||||
|
return new MultiValueDimensionSelector();
|
||||||
} else {
|
} else {
|
||||||
return new DimensionSelector()
|
class SingleValueDimensionSelector implements DimensionSelector, IdLookup
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
|
@ -538,6 +566,47 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
if (extractionFn == null) {
|
||||||
|
final int valueId = lookupId(value);
|
||||||
|
if (valueId >= 0) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
return column.getSingleValueRow(cursorOffset.getOffset()) == valueId;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Employ precomputed BitSet optimization
|
||||||
|
return makeValueMatcher(Predicates.equalTo(value));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
final BitSet predicateMatchingValueIds = DimensionSelectorUtils.makePredicateMatchingSet(
|
||||||
|
this,
|
||||||
|
predicate
|
||||||
|
);
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
int rowValueId = column.getSingleValueRow(cursorOffset.getOffset());
|
||||||
|
return predicateMatchingValueIds.get(rowValueId);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -551,6 +620,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
return extractionFn == null ? value : extractionFn.apply(value);
|
return extractionFn == null ? value : extractionFn.apply(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return extractionFn == null ? this : null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public int lookupId(String name)
|
||||||
{
|
{
|
||||||
|
@ -561,7 +643,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
return column.lookupId(name);
|
return column.lookupId(name);
|
||||||
}
|
}
|
||||||
};
|
}
|
||||||
|
return new SingleValueDimensionSelector();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -992,7 +1075,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
rowBitmap.iterator();
|
rowBitmap.iterator();
|
||||||
|
|
||||||
if (!iter.hasNext()) {
|
if (!iter.hasNext()) {
|
||||||
return new BooleanValueMatcher(false);
|
return BooleanValueMatcher.of(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (descending) {
|
if (descending) {
|
||||||
|
|
|
@ -19,14 +19,15 @@
|
||||||
|
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.base.Predicate;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
import io.druid.segment.data.SingleIndexedInt;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Map;
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
public class SingleScanTimeDimSelector implements DimensionSelector
|
public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
|
@ -35,7 +36,7 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
private final LongColumnSelector selector;
|
private final LongColumnSelector selector;
|
||||||
private final boolean descending;
|
private final boolean descending;
|
||||||
|
|
||||||
private final Map<Integer, String> timeValues = Maps.newHashMap();
|
private final List<String> timeValues = new ArrayList<>();
|
||||||
private String currentValue = null;
|
private String currentValue = null;
|
||||||
private long currentTimestamp = Long.MIN_VALUE;
|
private long currentTimestamp = Long.MIN_VALUE;
|
||||||
private int index = -1;
|
private int index = -1;
|
||||||
|
@ -58,6 +59,37 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
|
{
|
||||||
|
return new SingleIndexedInt(getDimensionValueIndex());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
return Objects.equals(lookupName(getDimensionValueIndex()), value);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
return predicate.apply(lookupName(getDimensionValueIndex()));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getDimensionValueIndex()
|
||||||
{
|
{
|
||||||
// if this the first timestamp, apply and cache extraction function result
|
// if this the first timestamp, apply and cache extraction function result
|
||||||
final long timestamp = selector.get();
|
final long timestamp = selector.get();
|
||||||
|
@ -65,7 +97,7 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
currentTimestamp = timestamp;
|
currentTimestamp = timestamp;
|
||||||
currentValue = extractionFn.apply(timestamp);
|
currentValue = extractionFn.apply(timestamp);
|
||||||
++index;
|
++index;
|
||||||
timeValues.put(index, currentValue);
|
timeValues.add(currentValue);
|
||||||
}
|
}
|
||||||
// if this is a new timestamp, apply and cache extraction function result
|
// if this is a new timestamp, apply and cache extraction function result
|
||||||
// since timestamps are assumed grouped and scanned once, we only need to
|
// since timestamps are assumed grouped and scanned once, we only need to
|
||||||
|
@ -85,7 +117,7 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
if (!Objects.equals(value, currentValue)) {
|
if (!Objects.equals(value, currentValue)) {
|
||||||
currentValue = value;
|
currentValue = value;
|
||||||
++index;
|
++index;
|
||||||
timeValues.put(index, currentValue);
|
timeValues.add(currentValue);
|
||||||
}
|
}
|
||||||
// Note: this could be further optimized by checking if the new value is one we have
|
// Note: this could be further optimized by checking if the new value is one we have
|
||||||
// previously seen, but would require keeping track of both the current and the maximum index
|
// previously seen, but would require keeping track of both the current and the maximum index
|
||||||
|
@ -93,39 +125,7 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
// otherwise, if the current timestamp is the same as the previous timestamp,
|
// otherwise, if the current timestamp is the same as the previous timestamp,
|
||||||
// keep using the same dimension value index
|
// keep using the same dimension value index
|
||||||
|
|
||||||
final int dimensionValueIndex = index;
|
return index;
|
||||||
return new IndexedInts()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public int size()
|
|
||||||
{
|
|
||||||
return 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int get(int i)
|
|
||||||
{
|
|
||||||
return dimensionValueIndex;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public IntIterator iterator()
|
|
||||||
{
|
|
||||||
return IntIterators.singleton(dimensionValueIndex);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void fill(int index, int[] toFill)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException("fill not supported");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException
|
|
||||||
{
|
|
||||||
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -145,8 +145,15 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("time column does not support lookups");
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,8 @@
|
||||||
package io.druid.segment;
|
package io.druid.segment;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
@ -29,19 +31,20 @@ import io.druid.collections.bitmap.MutableBitmap;
|
||||||
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
|
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
import io.druid.segment.data.ArrayBasedIndexedInts;
|
||||||
import io.druid.segment.data.Indexed;
|
import io.druid.segment.data.Indexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
|
import io.druid.segment.filter.BooleanValueMatcher;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||||
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
|
||||||
import it.unimi.dsi.fastutil.ints.IntList;
|
|
||||||
import it.unimi.dsi.fastutil.ints.IntLists;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import javax.annotation.Nullable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.BitSet;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -385,7 +388,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
final int dimIndex = desc.getIndex();
|
final int dimIndex = desc.getIndex();
|
||||||
final int maxId = getCardinality();
|
final int maxId = getCardinality();
|
||||||
|
|
||||||
return new DimensionSelector()
|
class IndexerDimensionSelector implements DimensionSelector, IdLookup
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
|
@ -399,59 +402,98 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
indices = null;
|
indices = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
IntList valsTmp = null;
|
int[] row = null;
|
||||||
|
int rowSize = 0;
|
||||||
if (indices == null || indices.length == 0) {
|
if (indices == null || indices.length == 0) {
|
||||||
final int nullId = getEncodedValue(null, false);
|
final int nullId = getEncodedValue(null, false);
|
||||||
if (nullId > -1) {
|
if (nullId > -1) {
|
||||||
if (nullId < maxId) {
|
if (nullId < maxId) {
|
||||||
valsTmp = IntLists.singleton(nullId);
|
row = new int[] {nullId};
|
||||||
|
rowSize = 1;
|
||||||
} else {
|
} else {
|
||||||
valsTmp = IntLists.EMPTY_LIST;
|
// Choose to use ArrayBasedIndexedInts later, instead of EmptyIndexedInts, for monomorphism
|
||||||
|
row = IntArrays.EMPTY_ARRAY;
|
||||||
|
rowSize = 0;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (valsTmp == null && indices != null && indices.length > 0) {
|
if (row == null && indices != null && indices.length > 0) {
|
||||||
valsTmp = new IntArrayList(indices.length);
|
row = new int[indices.length];
|
||||||
for (int i = 0; i < indices.length; i++) {
|
for (int id : indices) {
|
||||||
int id = indices[i];
|
|
||||||
if (id < maxId) {
|
if (id < maxId) {
|
||||||
valsTmp.add(id);
|
row[rowSize++] = id;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final IntList vals = valsTmp == null ? IntLists.EMPTY_LIST : valsTmp;
|
return ArrayBasedIndexedInts.of(row, rowSize);
|
||||||
return new IndexedInts()
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final String value)
|
||||||
|
{
|
||||||
|
if (extractionFn == null) {
|
||||||
|
final int valueId = lookupId(value);
|
||||||
|
if (valueId >= 0 || value == null) {
|
||||||
|
return new ValueMatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
Object[] dims = currEntry.getKey().getDims();
|
||||||
|
if (dimIndex >= dims.length) {
|
||||||
|
return value == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
int[] dimsInt = (int[]) dims[dimIndex];
|
||||||
|
if (dimsInt == null || dimsInt.length == 0) {
|
||||||
|
return value == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int id : dimsInt) {
|
||||||
|
if (id == valueId) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return BooleanValueMatcher.of(false);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Employ precomputed BitSet optimization
|
||||||
|
return makeValueMatcher(Predicates.equalTo(value));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
final BitSet predicateMatchingValueIds = DimensionSelectorUtils.makePredicateMatchingSet(this, predicate);
|
||||||
|
final boolean matchNull = predicate.apply(null);
|
||||||
|
return new ValueMatcher()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public boolean matches()
|
||||||
{
|
{
|
||||||
return vals.size();
|
Object[] dims = currEntry.getKey().getDims();
|
||||||
}
|
if (dimIndex >= dims.length) {
|
||||||
|
return matchNull;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
int[] dimsInt = (int[]) dims[dimIndex];
|
||||||
public int get(int index)
|
if (dimsInt == null || dimsInt.length == 0) {
|
||||||
{
|
return matchNull;
|
||||||
return vals.get(index);
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public IntIterator iterator()
|
|
||||||
{
|
|
||||||
return vals.iterator();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void fill(int index, int[] toFill)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException("fill not supported");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException
|
|
||||||
{
|
|
||||||
|
|
||||||
|
for (int id : dimsInt) {
|
||||||
|
if (predicateMatchingValueIds.get(id)) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -469,6 +511,19 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
return extractionFn == null ? strValue : extractionFn.apply(strValue);
|
return extractionFn == null ? strValue : extractionFn.apply(strValue);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return extractionFn == null ? this : null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public int lookupId(String name)
|
||||||
{
|
{
|
||||||
|
@ -479,7 +534,8 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
||||||
}
|
}
|
||||||
return getEncodedValue(name, false);
|
return getEncodedValue(name, false);
|
||||||
}
|
}
|
||||||
};
|
}
|
||||||
|
return new IndexerDimensionSelector();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,34 +19,66 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class ArrayBasedIndexedInts implements IndexedInts
|
public final class ArrayBasedIndexedInts implements IndexedInts
|
||||||
{
|
{
|
||||||
private final int[] expansion;
|
private static final ArrayBasedIndexedInts EMPTY = new ArrayBasedIndexedInts(IntArrays.EMPTY_ARRAY, 0);
|
||||||
|
|
||||||
public ArrayBasedIndexedInts(int[] expansion) {this.expansion = expansion;}
|
public static ArrayBasedIndexedInts of(int[] expansion)
|
||||||
|
{
|
||||||
|
if (expansion.length == 0) {
|
||||||
|
return EMPTY;
|
||||||
|
}
|
||||||
|
return new ArrayBasedIndexedInts(expansion, expansion.length);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ArrayBasedIndexedInts of(int[] expansion, int size)
|
||||||
|
{
|
||||||
|
if (size == 0) {
|
||||||
|
return EMPTY;
|
||||||
|
}
|
||||||
|
if (size < 0 || size > expansion.length) {
|
||||||
|
throw new IAE("Size[%s] should be between 0 and %s", size, expansion.length);
|
||||||
|
}
|
||||||
|
return new ArrayBasedIndexedInts(expansion, size);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final int[] expansion;
|
||||||
|
private final int size;
|
||||||
|
|
||||||
|
private ArrayBasedIndexedInts(int[] expansion, int size)
|
||||||
|
{
|
||||||
|
this.expansion = expansion;
|
||||||
|
this.size = size;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return expansion.length;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int get(int index)
|
public int get(int index)
|
||||||
{
|
{
|
||||||
|
if (index >= size) {
|
||||||
|
throw new IndexOutOfBoundsException("index: " + index + ", size: " + size);
|
||||||
|
}
|
||||||
return expansion[index];
|
return expansion[index];
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IntIterator iterator()
|
public IntIterator iterator()
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return IntIterators.wrap(expansion, 0, size);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,67 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public final class SingleIndexedInt implements IndexedInts
|
||||||
|
{
|
||||||
|
private final int value;
|
||||||
|
|
||||||
|
public SingleIndexedInt(int value)
|
||||||
|
{
|
||||||
|
this.value = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int get(int i)
|
||||||
|
{
|
||||||
|
if (i != 0) {
|
||||||
|
throw new IllegalArgumentException(i + " != 0");
|
||||||
|
}
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IntIterator iterator()
|
||||||
|
{
|
||||||
|
return IntIterators.singleton(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
|
@ -80,7 +80,7 @@ public class AndFilter implements BooleanFilter
|
||||||
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
|
||||||
{
|
{
|
||||||
if (filters.size() == 0) {
|
if (filters.size() == 0) {
|
||||||
return new BooleanValueMatcher(false);
|
return BooleanValueMatcher.of(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
final ValueMatcher[] matchers = new ValueMatcher[filters.size()];
|
final ValueMatcher[] matchers = new ValueMatcher[filters.size()];
|
||||||
|
|
|
@ -23,17 +23,14 @@ import io.druid.query.filter.ValueMatcher;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class BooleanValueMatcher implements ValueMatcher
|
public final class BooleanValueMatcher
|
||||||
{
|
{
|
||||||
private final boolean matches;
|
public static ValueMatcher of(boolean matches)
|
||||||
|
{
|
||||||
public BooleanValueMatcher(final boolean matches) {
|
return matches ? TrueValueMatcher.instance() : FalseValueMatcher.instance();
|
||||||
this.matches = matches;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private BooleanValueMatcher()
|
||||||
public boolean matches()
|
|
||||||
{
|
{
|
||||||
return matches;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,42 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.filter;
|
||||||
|
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
|
||||||
|
final class FalseValueMatcher implements ValueMatcher
|
||||||
|
{
|
||||||
|
private static final FalseValueMatcher INSTANCE = new FalseValueMatcher();
|
||||||
|
|
||||||
|
public static FalseValueMatcher instance()
|
||||||
|
{
|
||||||
|
return INSTANCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
private FalseValueMatcher()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
|
@ -283,12 +283,12 @@ public class Filters
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (Strings.isNullOrEmpty(value)) {
|
if (Strings.isNullOrEmpty(value)) {
|
||||||
return new BooleanValueMatcher(false);
|
return BooleanValueMatcher.of(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
final Long longValue = GuavaUtils.tryParseLong(value);
|
final Long longValue = GuavaUtils.tryParseLong(value);
|
||||||
if (longValue == null) {
|
if (longValue == null) {
|
||||||
return new BooleanValueMatcher(false);
|
return BooleanValueMatcher.of(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
return new ValueMatcher()
|
return new ValueMatcher()
|
||||||
|
|
|
@ -0,0 +1,42 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.filter;
|
||||||
|
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
|
|
||||||
|
final class TrueValueMatcher implements ValueMatcher
|
||||||
|
{
|
||||||
|
private static final TrueValueMatcher INSTANCE = new TrueValueMatcher();
|
||||||
|
|
||||||
|
public static TrueValueMatcher instance()
|
||||||
|
{
|
||||||
|
return INSTANCE;
|
||||||
|
}
|
||||||
|
|
||||||
|
private TrueValueMatcher()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean matches()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
|
@ -558,7 +558,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor)
|
private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor)
|
||||||
{
|
{
|
||||||
return filter == null
|
return filter == null
|
||||||
? new BooleanValueMatcher(true)
|
? BooleanValueMatcher.of(true)
|
||||||
: filter.makeMatcher(cursor);
|
: filter.makeMatcher(cursor);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.query.aggregation;
|
package io.druid.query.aggregation;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.js.JavaScriptConfig;
|
import io.druid.js.JavaScriptConfig;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
|
@ -34,11 +35,14 @@ import io.druid.query.filter.OrDimFilter;
|
||||||
import io.druid.query.filter.RegexDimFilter;
|
import io.druid.query.filter.RegexDimFilter;
|
||||||
import io.druid.query.filter.SearchQueryDimFilter;
|
import io.druid.query.filter.SearchQueryDimFilter;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.query.ordering.StringComparators;
|
import io.druid.query.ordering.StringComparators;
|
||||||
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||||
import io.druid.segment.ColumnSelectorFactory;
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
import io.druid.segment.FloatColumnSelector;
|
import io.druid.segment.FloatColumnSelector;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.LongColumnSelector;
|
import io.druid.segment.LongColumnSelector;
|
||||||
import io.druid.segment.ObjectColumnSelector;
|
import io.druid.segment.ObjectColumnSelector;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
|
@ -49,6 +53,7 @@ import io.druid.segment.data.IndexedInts;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
||||||
public class FilteredAggregatorTest
|
public class FilteredAggregatorTest
|
||||||
|
@ -99,12 +104,24 @@ public class FilteredAggregatorTest
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
{
|
{
|
||||||
if (selector.getIndex() % 3 == 2) {
|
if (selector.getIndex() % 3 == 2) {
|
||||||
return new ArrayBasedIndexedInts(new int[]{1});
|
return ArrayBasedIndexedInts.of(new int[]{1});
|
||||||
} else {
|
} else {
|
||||||
return new ArrayBasedIndexedInts(new int[]{0});
|
return ArrayBasedIndexedInts.of(new int[]{0});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(String value)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -125,16 +142,30 @@ public class FilteredAggregatorTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
switch (name) {
|
return true;
|
||||||
case "a":
|
}
|
||||||
return 0;
|
|
||||||
case "b":
|
@Nullable
|
||||||
return 1;
|
@Override
|
||||||
default:
|
public IdLookup idLookup()
|
||||||
throw new IllegalArgumentException();
|
{
|
||||||
}
|
return new IdLookup()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int lookupId(String name)
|
||||||
|
{
|
||||||
|
switch (name) {
|
||||||
|
case "a":
|
||||||
|
return 0;
|
||||||
|
case "b":
|
||||||
|
return 1;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.query.aggregation.cardinality;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
|
@ -41,7 +42,10 @@ import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||||
import it.unimi.dsi.fastutil.ints.IntIterators;
|
import it.unimi.dsi.fastutil.ints.IntIterators;
|
||||||
|
@ -157,6 +161,18 @@ public class CardinalityAggregatorTest
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(String value)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getValueCardinality()
|
public int getValueCardinality()
|
||||||
{
|
{
|
||||||
|
@ -171,9 +187,23 @@ public class CardinalityAggregatorTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String s)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
return ids.get(s);
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return new IdLookup()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int lookupId(String s)
|
||||||
|
{
|
||||||
|
return ids.get(s);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -134,8 +134,8 @@ public class ListFilteredDimensionSpecTest
|
||||||
Assert.assertEquals("c", selector.lookupName(0));
|
Assert.assertEquals("c", selector.lookupName(0));
|
||||||
Assert.assertEquals("g", selector.lookupName(1));
|
Assert.assertEquals("g", selector.lookupName(1));
|
||||||
|
|
||||||
Assert.assertEquals(0, selector.lookupId("c"));
|
Assert.assertEquals(0, selector.idLookup().lookupId("c"));
|
||||||
Assert.assertEquals(1, selector.lookupId("g"));
|
Assert.assertEquals(1, selector.idLookup().lookupId("g"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -158,8 +158,8 @@ public class ListFilteredDimensionSpecTest
|
||||||
Assert.assertEquals("a", selector.lookupName(0));
|
Assert.assertEquals("a", selector.lookupName(0));
|
||||||
Assert.assertEquals("z", selector.lookupName(23));
|
Assert.assertEquals("z", selector.lookupName(23));
|
||||||
|
|
||||||
Assert.assertEquals(0, selector.lookupId("a"));
|
Assert.assertEquals(0, selector.idLookup().lookupId("a"));
|
||||||
Assert.assertEquals(23, selector.lookupId("z"));
|
Assert.assertEquals(23, selector.idLookup().lookupId("z"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -186,7 +186,7 @@ public class ListFilteredDimensionSpecTest
|
||||||
Assert.assertEquals("a", selector.lookupName(0));
|
Assert.assertEquals("a", selector.lookupName(0));
|
||||||
Assert.assertEquals("z", selector.lookupName(24));
|
Assert.assertEquals("z", selector.lookupName(24));
|
||||||
|
|
||||||
Assert.assertEquals(0, selector.lookupId("a"));
|
Assert.assertEquals(0, selector.idLookup().lookupId("a"));
|
||||||
Assert.assertEquals(24, selector.lookupId("z"));
|
Assert.assertEquals(24, selector.idLookup().lookupId("z"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -96,7 +96,7 @@ public class RegexFilteredDimensionSpecTest
|
||||||
Assert.assertEquals("c", selector.lookupName(0));
|
Assert.assertEquals("c", selector.lookupName(0));
|
||||||
Assert.assertEquals("g", selector.lookupName(1));
|
Assert.assertEquals("g", selector.lookupName(1));
|
||||||
|
|
||||||
Assert.assertEquals(0, selector.lookupId("c"));
|
Assert.assertEquals(0, selector.idLookup().lookupId("c"));
|
||||||
Assert.assertEquals(1, selector.lookupId("g"));
|
Assert.assertEquals(1, selector.idLookup().lookupId("g"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,10 +19,16 @@
|
||||||
|
|
||||||
package io.druid.query.dimension;
|
package io.druid.query.dimension;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import io.druid.query.filter.ValueMatcher;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
|
import io.druid.segment.IdLookup;
|
||||||
import io.druid.segment.data.ArrayBasedIndexedInts;
|
import io.druid.segment.data.ArrayBasedIndexedInts;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test dimension selector that has cardinality=26
|
* Test dimension selector that has cardinality=26
|
||||||
* encoding 0 -> a, 1 -> b, ...
|
* encoding 0 -> a, 1 -> b, ...
|
||||||
|
@ -40,7 +46,19 @@ class TestDimensionSelector implements DimensionSelector
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts getRow()
|
public IndexedInts getRow()
|
||||||
{
|
{
|
||||||
return new ArrayBasedIndexedInts(new int[]{2, 4, 6});
|
return ArrayBasedIndexedInts.of(new int[]{2, 4, 6});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(String value)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -56,9 +74,22 @@ class TestDimensionSelector implements DimensionSelector
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int lookupId(String name)
|
public boolean nameLookupPossibleInAdvance()
|
||||||
{
|
{
|
||||||
return name.charAt(0) - 'a';
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return new IdLookup()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int lookupId(String name)
|
||||||
|
{
|
||||||
|
return name.charAt(0) - 'a';
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -53,8 +53,8 @@ public class NullDimensionSelectorTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLookupId() throws Exception {
|
public void testLookupId() throws Exception {
|
||||||
Assert.assertEquals(0, selector.lookupId(null));
|
Assert.assertEquals(0, selector.idLookup().lookupId(null));
|
||||||
Assert.assertEquals(0, selector.lookupId(""));
|
Assert.assertEquals(0, selector.idLookup().lookupId(""));
|
||||||
Assert.assertEquals(-1, selector.lookupId("billy"));
|
Assert.assertEquals(-1, selector.idLookup().lookupId("billy"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -123,7 +123,7 @@ public class CompressedVSizeIndexedV3WriterTest
|
||||||
@Override
|
@Override
|
||||||
public IndexedInts apply(@Nullable final int[] input)
|
public IndexedInts apply(@Nullable final int[] input)
|
||||||
{
|
{
|
||||||
return new ArrayBasedIndexedInts(input);
|
return ArrayBasedIndexedInts.of(input);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
), offsetChunkFactor, maxValue, byteOrder, compressionStrategy
|
), offsetChunkFactor, maxValue, byteOrder, compressionStrategy
|
||||||
|
|
Loading…
Reference in New Issue