mirror of https://github.com/apache/druid.git
Merge pull request #2267 from himanshug/fix_topn_multi_val_filter
Remap id's returned in XXXFilteredDimensionSpec.getRow() as per reduced cardinality
This commit is contained in:
commit
e0932ba1c2
|
@ -22,6 +22,13 @@ package io.druid.query.dimension;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
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;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -65,4 +72,51 @@ public abstract class BaseFilteredDimensionSpec implements DimensionSpec
|
|||
{
|
||||
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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,16 +21,14 @@ package io.druid.query.dimension;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.query.filter.DimFilterCacheHelper;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.ListBasedIndexedInts;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -40,12 +38,12 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
|
||||
private static final byte CACHE_TYPE_ID = 0x3;
|
||||
|
||||
private final List<String> values;
|
||||
private final Set<String> values;
|
||||
private final boolean isWhitelist;
|
||||
|
||||
public ListFilteredDimensionSpec(
|
||||
@JsonProperty("delegate") DimensionSpec delegate,
|
||||
@JsonProperty("values") List<String> values,
|
||||
@JsonProperty("values") Set<String> values,
|
||||
@JsonProperty("isWhitelist") Boolean isWhitelist
|
||||
)
|
||||
{
|
||||
|
@ -58,7 +56,7 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<String> getValues()
|
||||
public Set<String> getValues()
|
||||
{
|
||||
return values;
|
||||
}
|
||||
|
@ -76,55 +74,31 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
return selector;
|
||||
}
|
||||
|
||||
final Set<Integer> matched = new HashSet<>(values.size());
|
||||
for (String value : values) {
|
||||
int i = selector.lookupId(value);
|
||||
if (i >= 0) {
|
||||
matched.add(i);
|
||||
}
|
||||
};
|
||||
int selectorCardinality = selector.getValueCardinality();
|
||||
int cardinality = isWhitelist ? values.size() : selectorCardinality - values.size();
|
||||
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
IndexedInts baseRow = selector.getRow();
|
||||
List<Integer> result = new ArrayList<>(baseRow.size());
|
||||
int count = 0;
|
||||
final Map<Integer,Integer> forwardMapping = new HashMap<>(cardinality);
|
||||
final int[] reverseMapping = new int[cardinality];
|
||||
|
||||
for (int i : baseRow) {
|
||||
if (matched.contains(i)) {
|
||||
if (isWhitelist) {
|
||||
result.add(i);
|
||||
}
|
||||
} else {
|
||||
if (!isWhitelist) {
|
||||
result.add(i);
|
||||
}
|
||||
}
|
||||
if (isWhitelist) {
|
||||
for (String value : values) {
|
||||
int i = selector.lookupId(value);
|
||||
if (i >= 0) {
|
||||
forwardMapping.put(i, count);
|
||||
reverseMapping[count++] = i;
|
||||
}
|
||||
|
||||
return new ListBasedIndexedInts(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return matched.size();
|
||||
} else {
|
||||
for (int i = 0; i < selectorCardinality; i++) {
|
||||
if (!values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
||||
forwardMapping.put(i, count);
|
||||
reverseMapping[count++] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return selector.lookupName(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return selector.lookupId(name);
|
||||
}
|
||||
};
|
||||
return BaseFilteredDimensionSpec.decorate(selector, forwardMapping, reverseMapping);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,13 +25,10 @@ import com.google.common.base.Strings;
|
|||
import com.metamx.common.StringUtils;
|
||||
import io.druid.query.filter.DimFilterCacheHelper;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.ListBasedIndexedInts;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
|
@ -68,48 +65,24 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
return selector;
|
||||
}
|
||||
|
||||
final BitSet bitSetOfIds = new BitSet(selector.getValueCardinality());
|
||||
if (selector == null) {
|
||||
return selector;
|
||||
}
|
||||
|
||||
int count = 0;
|
||||
final Map<Integer,Integer> forwardMapping = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < selector.getValueCardinality(); i++) {
|
||||
if (compiledRegex.matcher(Strings.nullToEmpty(selector.lookupName(i))).matches()) {
|
||||
bitSetOfIds.set(i);
|
||||
forwardMapping.put(i, count++);
|
||||
}
|
||||
}
|
||||
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
IndexedInts baseRow = selector.getRow();
|
||||
List<Integer> result = new ArrayList<>(baseRow.size());
|
||||
|
||||
for (int i : baseRow) {
|
||||
if (bitSetOfIds.get(i)) {
|
||||
result.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new ListBasedIndexedInts(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return bitSetOfIds.cardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return selector.lookupName(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return selector.lookupId(name);
|
||||
}
|
||||
};
|
||||
final int[] reverseMapping = new int[forwardMapping.size()];
|
||||
for (Map.Entry<Integer, Integer> e : forwardMapping.entrySet()) {
|
||||
reverseMapping[e.getValue().intValue()] = e.getKey().intValue();
|
||||
}
|
||||
return BaseFilteredDimensionSpec.decorate(selector, forwardMapping, reverseMapping);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,9 @@ package io.druid.query;
|
|||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
|
@ -37,11 +39,18 @@ import io.druid.query.aggregation.AggregatorFactory;
|
|||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.dimension.ListFilteredDimensionSpec;
|
||||
import io.druid.query.dimension.RegexFilteredDimensionSpec;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.query.groupby.GroupByQueryRunnerTestHelper;
|
||||
import io.druid.query.spec.LegacySegmentSpec;
|
||||
import io.druid.query.topn.TopNQuery;
|
||||
import io.druid.query.topn.TopNQueryBuilder;
|
||||
import io.druid.query.topn.TopNQueryConfig;
|
||||
import io.druid.query.topn.TopNQueryQueryToolChest;
|
||||
import io.druid.query.topn.TopNQueryRunnerFactory;
|
||||
import io.druid.query.topn.TopNResultValue;
|
||||
import io.druid.segment.IncrementalIndexSegment;
|
||||
import io.druid.segment.IndexSpec;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
|
@ -50,6 +59,7 @@ import io.druid.segment.TestHelper;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -58,6 +68,7 @@ import java.io.File;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -250,6 +261,63 @@ public class MultiValuedDimensionTest
|
|||
TestHelper.assertExpectedObjects(expectedResults, Sequences.toList(result, new ArrayList<Row>()), "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithDimFilterAndWithFilteredDimSpec() throws Exception
|
||||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource("xx")
|
||||
.granularity(QueryGranularity.ALL)
|
||||
.dimension(new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("tags", "tags"),
|
||||
ImmutableSet.of("t3"),
|
||||
null
|
||||
))
|
||||
.metric("count")
|
||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||
.aggregators(
|
||||
Arrays.asList(
|
||||
new AggregatorFactory[]
|
||||
{
|
||||
new CountAggregatorFactory("count")
|
||||
}
|
||||
))
|
||||
.threshold(5)
|
||||
.filters(new SelectorDimFilter("tags", "t3")).build();
|
||||
|
||||
QueryRunnerFactory factory = new TopNQueryRunnerFactory(
|
||||
TestQueryRunners.getPool(),
|
||||
new TopNQueryQueryToolChest(
|
||||
new TopNQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
QueryRunner<Result<TopNResultValue>> runner = QueryRunnerTestHelper.makeQueryRunner(
|
||||
factory,
|
||||
new QueryableIndexSegment("sid1", queryableIndex)
|
||||
);
|
||||
Map<String, Object> context = Maps.newHashMap();
|
||||
Sequence<Result<TopNResultValue>> result = runner.run(query, context);
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"tags", "t3",
|
||||
"count", 2L
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
TestHelper.assertExpectedObjects(
|
||||
expectedResults,
|
||||
Sequences.toList(result, new ArrayList<Result<TopNResultValue>>()),
|
||||
""
|
||||
);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanup() throws Exception
|
||||
{
|
||||
|
|
|
@ -20,8 +20,10 @@
|
|||
package io.druid.query.dimension;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -54,7 +56,7 @@ public class ListFilteredDimensionSpecTest
|
|||
|
||||
ListFilteredDimensionSpec expected = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableList.of("xxx"),
|
||||
ImmutableSet.of("xxx"),
|
||||
true
|
||||
);
|
||||
|
||||
|
@ -78,7 +80,7 @@ public class ListFilteredDimensionSpecTest
|
|||
|
||||
expected = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableList.of("xxx"),
|
||||
ImmutableSet.of("xxx"),
|
||||
false
|
||||
);
|
||||
|
||||
|
@ -90,13 +92,13 @@ public class ListFilteredDimensionSpecTest
|
|||
{
|
||||
ListFilteredDimensionSpec spec1 = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableList.of("xxx"),
|
||||
ImmutableSet.of("xxx"),
|
||||
null
|
||||
);
|
||||
|
||||
ListFilteredDimensionSpec spec2 = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableList.of("xyz"),
|
||||
ImmutableSet.of("xyz"),
|
||||
null
|
||||
);
|
||||
|
||||
|
@ -104,10 +106,59 @@ public class ListFilteredDimensionSpecTest
|
|||
|
||||
ListFilteredDimensionSpec spec3 = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableList.of("xxx"),
|
||||
ImmutableSet.of("xxx"),
|
||||
false
|
||||
);
|
||||
|
||||
Assert.assertFalse(Arrays.equals(spec1.getCacheKey(), spec3.getCacheKey()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecoratorWithWhitelist()
|
||||
{
|
||||
ListFilteredDimensionSpec spec = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableSet.of("c", "g"),
|
||||
true
|
||||
);
|
||||
|
||||
DimensionSelector selector = spec.decorate(TestDimensionSelector.instance);
|
||||
|
||||
Assert.assertEquals(2, selector.getValueCardinality());
|
||||
|
||||
IndexedInts row = selector.getRow();
|
||||
Assert.assertEquals(2, row.size());
|
||||
Assert.assertEquals(0, row.get(0));
|
||||
Assert.assertEquals(1, row.get(1));
|
||||
|
||||
Assert.assertEquals("c", selector.lookupName(0));
|
||||
Assert.assertEquals("g", selector.lookupName(1));
|
||||
|
||||
Assert.assertEquals(0, selector.lookupId("c"));
|
||||
Assert.assertEquals(1, selector.lookupId("g"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecoratorWithBlacklist()
|
||||
{
|
||||
ListFilteredDimensionSpec spec = new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
ImmutableSet.of("c", "g"),
|
||||
false
|
||||
);
|
||||
|
||||
DimensionSelector selector = spec.decorate(TestDimensionSelector.instance);
|
||||
|
||||
Assert.assertEquals(24, selector.getValueCardinality());
|
||||
|
||||
IndexedInts row = selector.getRow();
|
||||
Assert.assertEquals(1, row.size());
|
||||
Assert.assertEquals(3, row.get(0));
|
||||
|
||||
Assert.assertEquals("a", selector.lookupName(0));
|
||||
Assert.assertEquals("z", selector.lookupName(23));
|
||||
|
||||
Assert.assertEquals(0, selector.lookupId("a"));
|
||||
Assert.assertEquals(23, selector.lookupId("z"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,9 @@
|
|||
package io.druid.query.dimension;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -73,4 +75,28 @@ public class RegexFilteredDimensionSpecTest
|
|||
|
||||
Assert.assertFalse(Arrays.equals(spec1.getCacheKey(), spec2.getCacheKey()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecorator()
|
||||
{
|
||||
RegexFilteredDimensionSpec spec = new RegexFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("foo", "bar"),
|
||||
"[c,g]"
|
||||
);
|
||||
|
||||
DimensionSelector selector = spec.decorate(TestDimensionSelector.instance);
|
||||
|
||||
Assert.assertEquals(2, selector.getValueCardinality());
|
||||
|
||||
IndexedInts row = selector.getRow();
|
||||
Assert.assertEquals(2, row.size());
|
||||
Assert.assertEquals(0, row.get(0));
|
||||
Assert.assertEquals(1, row.get(1));
|
||||
|
||||
Assert.assertEquals("c", selector.lookupName(0));
|
||||
Assert.assertEquals("g", selector.lookupName(1));
|
||||
|
||||
Assert.assertEquals(0, selector.lookupId("c"));
|
||||
Assert.assertEquals(1, selector.lookupId("g"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,64 @@
|
|||
/*
|
||||
* 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 io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.data.ArrayBasedIndexedInts;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
|
||||
/**
|
||||
* Test dimension selector that has cardinality=26
|
||||
* encoding 0 -> a, 1 -> b, ...
|
||||
* row -> [c,e,g]
|
||||
*/
|
||||
class TestDimensionSelector implements DimensionSelector
|
||||
{
|
||||
public final static TestDimensionSelector instance = new TestDimensionSelector();
|
||||
|
||||
private TestDimensionSelector()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return new ArrayBasedIndexedInts(new int[]{2, 4, 6});
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return 26;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return String.valueOf((char) (id + 'a'));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return name.charAt(0) - 'a';
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue