mirror of https://github.com/apache/druid.git
Merge pull request #995 from druid-io/null-fixes
Towards consistent null handling
This commit is contained in:
commit
ccebf281e6
|
@ -0,0 +1,2 @@
|
|||
Druid - a distributed column store.
|
||||
Copyright 2012-2015 Metamarkets Group Inc.
|
|
@ -499,9 +499,4 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
|
|||
return partitionHolder;
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args)
|
||||
{
|
||||
System.out.println(new Interval(new DateTime(), (DateTime) null));
|
||||
}
|
||||
}
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -587,6 +587,8 @@
|
|||
<argLine>-Duser.language=en -Duser.country=US</argLine>
|
||||
<systemPropertyVariables>
|
||||
<user.timezone>UTC</user.timezone>
|
||||
<user.country>US</user.country>
|
||||
<user.language>en</user.language>
|
||||
</systemPropertyVariables>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
|
@ -81,7 +81,6 @@
|
|||
<artifactId>mapdb</artifactId>
|
||||
</dependency>
|
||||
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
|
|
|
@ -26,15 +26,48 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "partial", value = MatchingDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "javascript", value = JavascriptDimExtractionFn.class)
|
||||
})
|
||||
/**
|
||||
* A DimExtractionFn is a function that can be used to modify the values of a dimension column.
|
||||
*
|
||||
* A simple example of the type of operation this enables is the RegexDimExtractionFn which applies a
|
||||
* regular expression with a capture group. When the regular expression matches the value of a dimension,
|
||||
* the value captured by the group is used for grouping operations instead of the dimension value.
|
||||
*/
|
||||
public interface DimExtractionFn
|
||||
{
|
||||
/**
|
||||
* Returns a byte[] unique to all concrete implementations of DimExtractionFn. This byte[] is used to
|
||||
* generate a cache key for the specific query.
|
||||
*
|
||||
* @return a byte[] unit to all concrete implements of DimExtractionFn
|
||||
*/
|
||||
public byte[] getCacheKey();
|
||||
|
||||
/**
|
||||
* The "extraction" function. This should map a dimension value into some other value.
|
||||
*
|
||||
* In order to maintain the "null and empty string are equivalent" semantics that Druid provides, the
|
||||
* empty string is considered invalid output for this method and should instead return null. This is
|
||||
* a contract on the method rather than enforced at a lower level in order to eliminate a global check
|
||||
* for extraction functions that do not already need one.
|
||||
*
|
||||
*
|
||||
* @param dimValue the original value of the dimension
|
||||
* @return a value that should be used instead of the original
|
||||
*/
|
||||
public String apply(String dimValue);
|
||||
|
||||
/**
|
||||
* Offers information on whether the extraction will preserve the original ordering of the values.
|
||||
*
|
||||
* Some optimizations of queries is possible if ordering is preserved. Null values *do* count towards
|
||||
* ordering.
|
||||
*
|
||||
* @return true if ordering is preserved, false otherwise
|
||||
*/
|
||||
public boolean preservesOrdering();
|
||||
}
|
||||
|
|
|
@ -20,6 +20,9 @@ package io.druid.query.extraction;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Strings;
|
||||
import com.metamx.common.StringUtils;
|
||||
import com.google.common.base.Strings;
|
||||
import com.metamx.common.StringUtils;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.ContextFactory;
|
||||
|
@ -89,7 +92,8 @@ public class JavascriptDimExtractionFn implements DimExtractionFn
|
|||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return fn.apply(dimValue);
|
||||
String retVal = fn.apply(dimValue);
|
||||
return Strings.isNullOrEmpty(retVal) ? null : retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.regex.Pattern;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class PartialDimExtractionFn implements DimExtractionFn
|
||||
public class MatchingDimExtractionFn implements DimExtractionFn
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x2;
|
||||
|
||||
|
@ -35,7 +35,7 @@ public class PartialDimExtractionFn implements DimExtractionFn
|
|||
private final Pattern pattern;
|
||||
|
||||
@JsonCreator
|
||||
public PartialDimExtractionFn(
|
||||
public MatchingDimExtractionFn(
|
||||
@JsonProperty("expr") String expr
|
||||
)
|
||||
{
|
||||
|
@ -70,7 +70,7 @@ public class PartialDimExtractionFn implements DimExtractionFn
|
|||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
|
@ -30,6 +30,5 @@ public interface BitmapIndexSelector
|
|||
public int getNumRows();
|
||||
public BitmapFactory getBitmapFactory();
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value);
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, int idx);
|
||||
public ImmutableRTree getSpatialIndex(String dimension);
|
||||
}
|
||||
|
|
|
@ -101,6 +101,11 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
|||
return postAggregatorSpecs;
|
||||
}
|
||||
|
||||
public boolean isSkipEmptyBuckets()
|
||||
{
|
||||
return Boolean.parseBoolean(getContextValue("skipEmptyBuckets", "false"));
|
||||
}
|
||||
|
||||
public TimeseriesQuery withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
|
||||
{
|
||||
return new TimeseriesQuery(
|
||||
|
|
|
@ -49,12 +49,18 @@ public class TimeseriesQueryEngine
|
|||
query.getGranularity(),
|
||||
new Function<Cursor, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
private final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
|
||||
private final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
|
||||
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(Cursor cursor)
|
||||
{
|
||||
Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs);
|
||||
|
||||
if (skipEmptyBuckets && cursor.isDone()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
while (!cursor.isDone()) {
|
||||
for (Aggregator aggregator : aggregators) {
|
||||
|
|
|
@ -103,10 +103,6 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
|
|||
Aggregator[] theAggregators = rowSelector[dimIndex];
|
||||
if (theAggregators == null) {
|
||||
String key = query.getDimensionSpec().getDimExtractionFn().apply(dimSelector.lookupName(dimIndex));
|
||||
if (key == null) {
|
||||
rowSelector[dimIndex] = EMPTY_ARRAY;
|
||||
continue;
|
||||
}
|
||||
theAggregators = aggregatesStore.get(key);
|
||||
if (theAggregators == null) {
|
||||
theAggregators = makeAggregators(cursor, query.getAggregatorSpecs());
|
||||
|
|
|
@ -277,7 +277,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
)
|
||||
);
|
||||
|
||||
return new Result<TopNResultValue>(
|
||||
return new Result<>(
|
||||
result.getTimestamp(),
|
||||
new TopNResultValue(serializedValues)
|
||||
);
|
||||
|
@ -400,7 +400,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
retVal.add(vals);
|
||||
}
|
||||
|
||||
return new Result<TopNResultValue>(timestamp, new TopNResultValue(retVal));
|
||||
return new Result<>(timestamp, new TopNResultValue(retVal));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -408,7 +408,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
@Override
|
||||
public Sequence<Result<TopNResultValue>> mergeSequences(Sequence<Sequence<Result<TopNResultValue>>> seqOfSequences)
|
||||
{
|
||||
return new MergeSequence<Result<TopNResultValue>>(getOrdering(), seqOfSequences);
|
||||
return new MergeSequence<>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -416,10 +416,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
@Override
|
||||
public QueryRunner<Result<TopNResultValue>> preMergeQueryDecoration(QueryRunner<Result<TopNResultValue>> runner)
|
||||
{
|
||||
return new IntervalChunkingQueryRunner<Result<TopNResultValue>>(
|
||||
runner,
|
||||
config.getChunkPeriod()
|
||||
);
|
||||
return new IntervalChunkingQueryRunner<>(runner, config.getChunkPeriod());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -485,7 +482,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
@Override
|
||||
public Result<TopNResultValue> apply(Result<TopNResultValue> input)
|
||||
{
|
||||
return new Result<TopNResultValue>(
|
||||
return new Result<>(
|
||||
input.getTimestamp(),
|
||||
new TopNResultValue(
|
||||
Lists.<Object>newArrayList(
|
||||
|
@ -505,7 +502,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
);
|
||||
}
|
||||
|
||||
return new Result<TopNResultValue>(
|
||||
return new Result<>(
|
||||
input.getTimestamp(),
|
||||
new TopNResultValue(
|
||||
Lists.<Object>newArrayList(
|
||||
|
|
|
@ -82,7 +82,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory<Result<TopNRes
|
|||
ExecutorService queryExecutor, Iterable<QueryRunner<Result<TopNResultValue>>> queryRunners
|
||||
)
|
||||
{
|
||||
return new ChainedExecutionQueryRunner<Result<TopNResultValue>>(
|
||||
return new ChainedExecutionQueryRunner<>(
|
||||
queryExecutor, toolchest.getOrdering(), queryWatcher, queryRunners
|
||||
);
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.metamx.collections.bitmap.BitmapFactory;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
|
@ -112,8 +113,13 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
{
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null) {
|
||||
return bitmapFactory.makeEmptyImmutableBitmap();
|
||||
if (Strings.isNullOrEmpty(value)) {
|
||||
return bitmapFactory.complement(bitmapFactory.makeEmptyImmutableBitmap(), getNumRows());
|
||||
} else {
|
||||
return bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
}
|
||||
|
||||
if (!column.getCapabilities().hasBitmapIndexes()) {
|
||||
bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
|
@ -121,23 +127,6 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
return column.getBitmapIndex().getBitmap(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, int idx)
|
||||
{
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null || column.getCapabilities() == null) {
|
||||
bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
if (!column.getCapabilities().hasBitmapIndexes()) {
|
||||
bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
|
||||
// This is a workaround given the current state of indexing, I feel shame
|
||||
final int index1 = column.getBitmapIndex().hasNulls() ? idx + 1 : idx;
|
||||
|
||||
return column.getBitmapIndex().getBitmap(index1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* 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.Strings;
|
||||
import com.google.common.collect.Iterators;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
public class NullDimensionSelector implements DimensionSelector
|
||||
{
|
||||
|
||||
private static final IndexedInts SINGLETON = new IndexedInts() {
|
||||
@Override
|
||||
public int size() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator() {
|
||||
return Iterators.singletonIterator(0);
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return SINGLETON;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return Strings.isNullOrEmpty(name) ? 0 : -1;
|
||||
}
|
||||
}
|
|
@ -49,6 +49,8 @@ import java.util.Map;
|
|||
*/
|
||||
public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||
{
|
||||
private static final NullDimensionSelector NULL_DIMENSION_SELECTOR = new NullDimensionSelector();
|
||||
|
||||
private final QueryableIndex index;
|
||||
|
||||
public QueryableIndexStorageAdapter(
|
||||
|
@ -274,8 +276,11 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
{
|
||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension);
|
||||
final Column columnDesc = index.getColumn(dimension);
|
||||
if (columnDesc == null) {
|
||||
return NULL_DIMENSION_SELECTOR;
|
||||
}
|
||||
|
||||
if (cachedColumn == null && columnDesc != null) {
|
||||
if (cachedColumn == null) {
|
||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||
dictionaryColumnCache.put(dimension, cachedColumn);
|
||||
}
|
||||
|
@ -283,8 +288,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
final DictionaryEncodedColumn column = cachedColumn;
|
||||
|
||||
if (column == null) {
|
||||
return null;
|
||||
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
|
||||
return NULL_DIMENSION_SELECTOR;
|
||||
}
|
||||
else if (columnDesc.getCapabilities().hasMultipleValues()) {
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.Filter;
|
||||
|
@ -59,9 +60,9 @@ public class SelectorFilter implements Filter
|
|||
{
|
||||
final DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimension);
|
||||
|
||||
// Missing columns are treated the same way as selector.getBitmapIndex, always returning false
|
||||
// Missing columns match a null or empty string value and don't match anything else
|
||||
if (dimensionSelector == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
return new BooleanValueMatcher(Strings.isNullOrEmpty(value));
|
||||
} else {
|
||||
final int valueId = dimensionSelector.lookupId(value);
|
||||
return new ValueMatcher()
|
||||
|
|
|
@ -579,6 +579,9 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
|||
if (dim != null && dim.length != 0) {
|
||||
theVals.put(dimensions.get(i), dim.length == 1 ? dim[0] : Arrays.asList(dim));
|
||||
}
|
||||
else {
|
||||
theVals.put(dimensions.get(i), null);
|
||||
}
|
||||
}
|
||||
|
||||
AggregatorType[] aggs = getAggsForRow(rowOffset);
|
||||
|
|
|
@ -20,6 +20,7 @@ package io.druid.segment.incremental;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.collections.spatial.search.Bound;
|
||||
|
@ -35,6 +36,7 @@ import io.druid.segment.Cursor;
|
|||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.FloatColumnSelector;
|
||||
import io.druid.segment.LongColumnSelector;
|
||||
import io.druid.segment.NullDimensionSelector;
|
||||
import io.druid.segment.ObjectColumnSelector;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.column.Column;
|
||||
|
@ -59,6 +61,8 @@ import java.util.concurrent.ConcurrentNavigableMap;
|
|||
public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||
{
|
||||
private static final Splitter SPLITTER = Splitter.on(",");
|
||||
private static final NullDimensionSelector NULL_DIMENSION_SELECTOR = new NullDimensionSelector();
|
||||
|
||||
private final IncrementalIndex index;
|
||||
|
||||
public IncrementalIndexStorageAdapter(
|
||||
|
@ -272,7 +276,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
{
|
||||
final IncrementalIndex.DimDim dimValLookup = index.getDimension(dimension);
|
||||
if (dimValLookup == null) {
|
||||
return null;
|
||||
return NULL_DIMENSION_SELECTOR;
|
||||
}
|
||||
|
||||
final int maxId = dimValLookup.size();
|
||||
|
@ -447,18 +451,24 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public Object get()
|
||||
{
|
||||
final String[][] dims = currEntry.getKey().getDims();
|
||||
IncrementalIndex.TimeAndDims key = currEntry.getKey();
|
||||
if (key == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String[][] dims = key.getDims();
|
||||
if (dimensionIndex >= dims.length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final String[] dimVals = dims[dimensionIndex];
|
||||
if (dimVals == null || dimVals.length == 0) {
|
||||
return null;
|
||||
}
|
||||
if (dimVals.length == 1) {
|
||||
return dimVals[0];
|
||||
} else if (dimVals.length == 0) {
|
||||
return null;
|
||||
} else {
|
||||
return dimVals;
|
||||
}
|
||||
return dimVals;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -490,7 +500,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
public void set(Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry)
|
||||
{
|
||||
this.currEntry = currEntry;
|
||||
this.currEntry = currEntry;
|
||||
}
|
||||
|
||||
public IncrementalIndex.TimeAndDims getKey()
|
||||
|
@ -520,7 +529,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
{
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
||||
if (dimIndexObject == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
return new BooleanValueMatcher(Strings.isNullOrEmpty(value));
|
||||
}
|
||||
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
||||
if (!dimDim.contains(value)) {
|
||||
|
|
|
@ -19,7 +19,7 @@ package io.druid.query.extraction.extraction;
|
|||
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.query.extraction.DimExtractionFn;
|
||||
import io.druid.query.extraction.PartialDimExtractionFn;
|
||||
import io.druid.query.extraction.MatchingDimExtractionFn;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -29,7 +29,7 @@ import java.util.Set;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class PartialDimExtractionFnTest
|
||||
public class MatchingDimExtractionFnTest
|
||||
{
|
||||
private static final String[] testStrings = {
|
||||
"Quito",
|
||||
|
@ -47,7 +47,7 @@ public class PartialDimExtractionFnTest
|
|||
public void testExtraction()
|
||||
{
|
||||
String regex = ".*[Tt][Oo].*";
|
||||
DimExtractionFn dimExtractionFn = new PartialDimExtractionFn(regex);
|
||||
DimExtractionFn dimExtractionFn = new MatchingDimExtractionFn(regex);
|
||||
List<String> expected = Arrays.asList("Quito", "Tokyo", "Stockholm", "Pretoria", "Wellington");
|
||||
Set<String> extracted = Sets.newHashSet();
|
||||
|
|
@ -50,6 +50,7 @@ import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
|||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||
import io.druid.query.extraction.DimExtractionFn;
|
||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||
import io.druid.query.filter.JavaScriptDimFilter;
|
||||
import io.druid.query.filter.RegexDimFilter;
|
||||
|
@ -98,7 +99,7 @@ public class GroupByQueryRunnerTest
|
|||
public static Collection<?> constructorFeeder() throws IOException
|
||||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
final StupidPool<ByteBuffer> pool = new StupidPool<ByteBuffer>(
|
||||
final StupidPool<ByteBuffer> pool = new StupidPool<>(
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
|
@ -183,7 +184,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -225,7 +226,7 @@ public class GroupByQueryRunnerTest
|
|||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
QueryRunnerTestHelper.qualityUniques
|
||||
)
|
||||
|
@ -255,7 +256,7 @@ public class GroupByQueryRunnerTest
|
|||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
QueryRunnerTestHelper.qualityCardinality
|
||||
)
|
||||
|
@ -278,52 +279,132 @@ public class GroupByQueryRunnerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithDimExtractionFn()
|
||||
public void testGroupByWithNullProducingDimExtractionFn()
|
||||
{
|
||||
final DimExtractionFn fn1 = new RegexDimExtractionFn("(\\w{1})");
|
||||
final DimExtractionFn nullExtractionFn = new DimExtractionFn()
|
||||
{
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[]{(byte) 0xFF};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return dimValue.equals("mezzanine") ? null : fn1.apply(dimValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(
|
||||
Lists.<DimensionSpec>newArrayList(
|
||||
new ExtractionDimensionSpec(
|
||||
"quality",
|
||||
"alias",
|
||||
new RegexDimExtractionFn("(\\w{1})")
|
||||
)
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
))
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new ExtractionDimensionSpec("quality", "alias", nullExtractionFn)
|
||||
))
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", null, "rows", 3L, "idx", 2870L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "m", "rows", 3L, "idx", 2870L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L),
|
||||
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", null, "rows", 3L, "idx", 2447L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "m", "rows", 3L, "idx", 2447L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L)
|
||||
);
|
||||
|
||||
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
TestHelper.assertExpectedObjects(expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, query), "");
|
||||
}
|
||||
|
||||
@Test
|
||||
/**
|
||||
* This test exists only to show what the current behavior is and not necessarily to define that this is
|
||||
* correct behavior. In fact, the behavior when returning the empty string from a DimExtractionFn is, by
|
||||
* contract, undefined, so this can do anything.
|
||||
*/
|
||||
public void testGroupByWithEmptyStringProducingDimExtractionFn()
|
||||
{
|
||||
final DimExtractionFn fn1 = new RegexDimExtractionFn("(\\w{1})");
|
||||
final DimExtractionFn emptyStringExtractionFn = new DimExtractionFn()
|
||||
{
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[]{(byte) 0xFF};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return dimValue.equals("mezzanine") ? "" : fn1.apply(dimValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
))
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new ExtractionDimensionSpec("quality", "alias", emptyStringExtractionFn)
|
||||
))
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "", "rows", 3L, "idx", 2870L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "a", "rows", 1L, "idx", 135L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "b", "rows", 1L, "idx", 118L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "e", "rows", 1L, "idx", 158L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "h", "rows", 1L, "idx", 120L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "n", "rows", 1L, "idx", 121L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "p", "rows", 3L, "idx", 2900L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "t", "rows", 2L, "idx", 197L),
|
||||
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "", "rows", 3L, "idx", 2447L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "a", "rows", 1L, "idx", 147L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "b", "rows", 1L, "idx", 112L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "e", "rows", 1L, "idx", 166L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "h", "rows", 1L, "idx", 113L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "n", "rows", 1L, "idx", 114L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "p", "rows", 3L, "idx", 2505L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "t", "rows", 2L, "idx", 223L)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedObjects(expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, query), "");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -343,7 +424,7 @@ public class GroupByQueryRunnerTest
|
|||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory(
|
||||
"idx",
|
||||
|
@ -539,7 +620,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -550,13 +631,11 @@ public class GroupByQueryRunnerTest
|
|||
final GroupByQuery allGranQuery = builder.copy().setGranularity(QueryGranularity.ALL).build();
|
||||
|
||||
QueryRunner mergedRunner = factory.getToolchest().mergeResults(
|
||||
new QueryRunner<Row>()
|
||||
{
|
||||
new QueryRunner<Row>() {
|
||||
@Override
|
||||
public Sequence<Row> run(
|
||||
Query<Row> query, Map<String, Object> responseContext
|
||||
)
|
||||
{
|
||||
) {
|
||||
// simulate two daily segments
|
||||
final Query query1 = query.withQuerySegmentSpec(
|
||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
|
||||
|
@ -617,7 +696,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -656,7 +735,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -735,7 +814,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -778,7 +857,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -819,7 +898,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -859,7 +938,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new DoubleSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1085,7 +1164,7 @@ public class GroupByQueryRunnerTest
|
|||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1150,7 +1229,7 @@ public class GroupByQueryRunnerTest
|
|||
|
||||
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
||||
configSupplier,
|
||||
new StupidPool<ByteBuffer>(
|
||||
new StupidPool<>(
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
|
@ -1191,22 +1270,22 @@ public class GroupByQueryRunnerTest
|
|||
final GroupByQuery query = builder.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "automotive", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "business", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "entertainment", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "health", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "mezzanine", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "news", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "premium", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "technology", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "index", null, "quality", "travel", "rows", 2L)
|
||||
);
|
||||
|
||||
Map<String, Object> context = Maps.newHashMap();
|
||||
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
|
||||
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
||||
configSupplier,
|
||||
new StupidPool<ByteBuffer>(
|
||||
new StupidPool<>(
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
|
@ -1246,22 +1325,22 @@ public class GroupByQueryRunnerTest
|
|||
final GroupByQuery query = builder.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "business", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "entertainment", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "health", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "mezzanine", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "news", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "premium", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "automotive", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "business", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "entertainment", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "health", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "mezzanine", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "news", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "premium", "rows", 6L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "technology", "rows", 2L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "billy", null, "quality", "travel", "rows", 2L)
|
||||
);
|
||||
|
||||
Map<String, Object> context = Maps.newHashMap();
|
||||
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
|
||||
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
||||
configSupplier,
|
||||
new StupidPool<ByteBuffer>(
|
||||
new StupidPool<>(
|
||||
new Supplier<ByteBuffer>()
|
||||
{
|
||||
@Override
|
||||
|
@ -1293,7 +1372,7 @@ public class GroupByQueryRunnerTest
|
|||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1351,7 +1430,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1389,7 +1468,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1426,7 +1505,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.emptyInterval)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
|
@ -1460,7 +1539,7 @@ public class GroupByQueryRunnerTest
|
|||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx_subagg", "index")
|
||||
)
|
||||
|
@ -1468,7 +1547,7 @@ public class GroupByQueryRunnerTest
|
|||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx_subpostagg", "+", Arrays.<PostAggregator>asList(
|
||||
"idx_subpostagg", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"),
|
||||
new ConstantPostAggregator("thousand", 1000)
|
||||
)
|
||||
|
@ -1722,7 +1801,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("alias", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
new LongSumAggregatorFactory("rows", "rows"),
|
||||
new LongSumAggregatorFactory("idx", "idx_subpostagg"),
|
||||
new DoubleSumAggregatorFactory("js_outer_agg", "js_agg")
|
||||
|
@ -1825,7 +1904,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index"),
|
||||
new HyperUniquesAggregatorFactory("quality_uniques", "quality_uniques")
|
||||
|
@ -1840,7 +1919,7 @@ public class GroupByQueryRunnerTest
|
|||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("alias", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
Arrays.asList(
|
||||
new LongSumAggregatorFactory("rows", "rows"),
|
||||
new LongSumAggregatorFactory("idx", "idx"),
|
||||
new HyperUniquesAggregatorFactory("uniq", "quality_uniques")
|
||||
|
|
|
@ -40,6 +40,7 @@ import io.druid.query.filter.AndDimFilter;
|
|||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.NotDimFilter;
|
||||
import io.druid.query.filter.RegexDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import io.druid.segment.TestHelper;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -54,6 +55,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -271,13 +273,13 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 6619L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
|
@ -318,13 +320,13 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-03-31", DateTimeZone.forID("America/Los_Angeles")),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 6619L)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T", DateTimeZone.forID("America/Los_Angeles")),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L)
|
||||
|
@ -366,7 +368,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
|
@ -403,7 +405,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults2 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
|
@ -450,13 +452,13 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-06T00:00:00.000-08:00", DateTimeZone.forID("America/Los_Angeles")),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 6071L)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-13T00:00:00.000-08:00", DateTimeZone.forID("America/Los_Angeles")),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 91L, "idx", 33382L)
|
||||
|
@ -571,7 +573,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-14T23:01Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 4717L)
|
||||
|
@ -613,7 +615,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
|
@ -650,7 +652,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults2 = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of("rows", 13L, "idx", 5827L, "uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
|
@ -719,7 +721,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -730,7 +732,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -769,7 +771,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -780,7 +782,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -819,7 +821,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -830,7 +832,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -869,7 +871,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -880,7 +882,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -919,7 +921,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -930,7 +932,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -977,7 +979,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -988,7 +990,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1035,7 +1037,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1046,7 +1048,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1093,7 +1095,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1104,7 +1106,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1149,7 +1151,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1160,7 +1162,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1211,7 +1213,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1222,7 +1224,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1255,7 +1257,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1266,7 +1268,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1286,6 +1288,116 @@ public class TimeseriesQueryRunnerTest
|
|||
TestHelper.assertExpectedResults(expectedResults, results);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeseriesWithFilterOnNonExistentDimensionSkipBuckets()
|
||||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.dayGran)
|
||||
.filters("bobby", "billy")
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", "true"))
|
||||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
|
||||
|
||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||
runner.run(query, new HashMap<String, Object>()),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
TestHelper.assertExpectedResults(expectedResults, results);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeseriesWithNullFilterOnNonExistentDimension()
|
||||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.dayGran)
|
||||
.filters("bobby", null)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"rows", 13L,
|
||||
"index", 6626.151596069336,
|
||||
"addRowsIndexConstant", 6640.151596069336,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
)
|
||||
),
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"rows", 13L,
|
||||
"index", 5833.2095947265625,
|
||||
"addRowsIndexConstant", 5847.2095947265625,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||
runner.run(query, new HashMap<String, Object>()),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
TestHelper.assertExpectedResults(expectedResults, results);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeseriesWithInvertedFilterOnNonExistentDimension()
|
||||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.dayGran)
|
||||
.filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally")))
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"rows", 13L,
|
||||
"index", 6626.151596069336,
|
||||
"addRowsIndexConstant", 6640.151596069336,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
)
|
||||
),
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"rows", 13L,
|
||||
"index", 5833.2095947265625,
|
||||
"addRowsIndexConstant", 5847.2095947265625,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||
runner.run(query, new HashMap<String, Object>()),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
TestHelper.assertExpectedResults(expectedResults, results);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeseriesWithNonExistentFilter()
|
||||
{
|
||||
|
@ -1299,7 +1411,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1310,7 +1422,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1357,7 +1469,7 @@ public class TimeseriesQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1368,7 +1480,7 @@ public class TimeseriesQueryRunnerTest
|
|||
)
|
||||
)
|
||||
),
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-02"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1691,7 +1803,7 @@ public class TimeseriesQueryRunnerTest
|
|||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1740,7 +1852,7 @@ public class TimeseriesQueryRunnerTest
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1789,11 +1901,11 @@ public class TimeseriesQueryRunnerTest
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"filteredAgg", 0L,
|
||||
"filteredAgg", 26L,
|
||||
"addRowsIndexConstant", 12486.361190795898d,
|
||||
"index", 12459.361190795898d,
|
||||
"uniques", 9.019833517963864d,
|
||||
|
@ -1839,7 +1951,7 @@ public class TimeseriesQueryRunnerTest
|
|||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
@ -1857,7 +1969,7 @@ public class TimeseriesQueryRunnerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testTimeSeriesWithFilteredAggNullValue()
|
||||
public void testTimeSeriesWithFilteredAggInvertedNullValue()
|
||||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
|
@ -1889,7 +2001,7 @@ public class TimeseriesQueryRunnerTest
|
|||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
);
|
||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
|
|
|
@ -19,9 +19,11 @@ package io.druid.query.topn;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.collections.StupidPool;
|
||||
|
@ -35,8 +37,10 @@ import io.druid.query.aggregation.AggregatorFactory;
|
|||
import io.druid.query.aggregation.MaxAggregatorFactory;
|
||||
import io.druid.query.aggregation.MinAggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||
import io.druid.query.extraction.DimExtractionFn;
|
||||
import io.druid.query.extraction.RegexDimExtractionFn;
|
||||
import io.druid.query.filter.AndDimFilter;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
|
@ -55,6 +59,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -822,13 +827,13 @@ public class TopNQueryRunnerTest
|
|||
.fields(
|
||||
Lists.<DimFilter>newArrayList(
|
||||
Druids.newSelectorDimFilterBuilder()
|
||||
.dimension(marketDimension)
|
||||
.value("billyblank")
|
||||
.build(),
|
||||
.dimension(marketDimension)
|
||||
.value("billyblank")
|
||||
.build(),
|
||||
Druids.newSelectorDimFilterBuilder()
|
||||
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||
.value("mezzanine")
|
||||
.build()
|
||||
.dimension(QueryRunnerTestHelper.qualityDimension)
|
||||
.value("mezzanine")
|
||||
.build()
|
||||
)
|
||||
).build();
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
|
@ -941,7 +946,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -983,7 +988,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1032,7 +1037,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
final ArrayList<Result<TopNResultValue>> expectedResults = Lists.newArrayList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1072,6 +1077,110 @@ public class TopNQueryRunnerTest
|
|||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithNonExistentDimension()
|
||||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.dimension("doesn't exist")
|
||||
.metric(QueryRunnerTestHelper.indexMetric)
|
||||
.threshold(1)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put("doesn't exist", null);
|
||||
put("rows", 26L);
|
||||
put("index", 12459.361190795898D);
|
||||
put("addRowsIndexConstant", 12486.361190795898D);
|
||||
put("uniques", QueryRunnerTestHelper.UNIQUES_9);
|
||||
}}
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithNonExistentDimensionAndActualFilter()
|
||||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.filters(QueryRunnerTestHelper.marketDimension, "upfront")
|
||||
.dimension("doesn't exist")
|
||||
.metric(QueryRunnerTestHelper.indexMetric)
|
||||
.threshold(4)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put("doesn't exist", null);
|
||||
put("rows", 4L);
|
||||
put("index", 4875.669677734375D);
|
||||
put("addRowsIndexConstant", 4880.669677734375D);
|
||||
put("uniques", QueryRunnerTestHelper.UNIQUES_2);
|
||||
}}
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithNonExistentDimensionAndNonExistentFilter()
|
||||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.filters("doesn't exist", null)
|
||||
.dimension("doesn't exist")
|
||||
.metric(QueryRunnerTestHelper.indexMetric)
|
||||
.threshold(1)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put("doesn't exist", null);
|
||||
put("rows", 26L);
|
||||
put("index", 12459.361190795898D);
|
||||
put("addRowsIndexConstant", 12486.361190795898D);
|
||||
put("uniques", QueryRunnerTestHelper.UNIQUES_9);
|
||||
}}
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNLexicographic()
|
||||
{
|
||||
|
@ -1087,7 +1196,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1135,7 +1244,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1176,7 +1285,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1303,7 +1412,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1572,8 +1681,156 @@ public class TopNQueryRunnerTest
|
|||
)
|
||||
)
|
||||
);
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNWithNullProducingDimExtractionFn()
|
||||
{
|
||||
final DimExtractionFn nullStringDimExtraction = new DimExtractionFn()
|
||||
{
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[]{(byte) 0xFF};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return dimValue.equals("total_market") ? null : dimValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
final TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.metric("rows")
|
||||
.threshold(4)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.dimension(
|
||||
new ExtractionDimensionSpec(marketDimension, marketDimension, nullStringDimExtraction)
|
||||
)
|
||||
.build();
|
||||
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
marketDimension, "spot",
|
||||
"rows", 18L,
|
||||
"index", 2231.8768157958984D,
|
||||
"addRowsIndexConstant", 2250.8768157958984D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
),
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put(marketDimension, null);
|
||||
put("rows", 4L);
|
||||
put("index", 5351.814697265625D);
|
||||
put("addRowsIndexConstant", 5356.814697265625D);
|
||||
put("uniques", QueryRunnerTestHelper.UNIQUES_2);
|
||||
}},
|
||||
ImmutableMap.<String, Object>of(
|
||||
marketDimension, "upfront",
|
||||
"rows", 4L,
|
||||
"index", 4875.669677734375D,
|
||||
"addRowsIndexConstant", 4880.669677734375D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_2
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
/**
|
||||
* This test exists only to show what the current behavior is and not necessarily to define that this is
|
||||
* correct behavior. In fact, the behavior when returning the empty string from a DimExtractionFn is, by
|
||||
* contract, undefined, so this can do anything.
|
||||
*/
|
||||
public void testTopNWithEmptyStringProducingDimExtractionFn()
|
||||
{
|
||||
final DimExtractionFn emptyStringDimExtraction = new DimExtractionFn()
|
||||
{
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[]{(byte) 0xFF};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return dimValue.equals("total_market") ? "" : dimValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean preservesOrdering()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
final TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.metric("rows")
|
||||
.threshold(4)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.dimension(
|
||||
new ExtractionDimensionSpec(marketDimension, marketDimension, emptyStringDimExtraction))
|
||||
.build();
|
||||
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
marketDimension, "spot",
|
||||
"rows", 18L,
|
||||
"index", 2231.8768157958984D,
|
||||
"addRowsIndexConstant", 2250.8768157958984D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_9
|
||||
),
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put(marketDimension, "");
|
||||
put("rows", 4L);
|
||||
put("index", 5351.814697265625D);
|
||||
put("addRowsIndexConstant", 5356.814697265625D);
|
||||
put("uniques", QueryRunnerTestHelper.UNIQUES_2);
|
||||
}},
|
||||
ImmutableMap.<String, Object>of(
|
||||
marketDimension, "upfront",
|
||||
"rows", 4L,
|
||||
"index", 4875.669677734375D,
|
||||
"addRowsIndexConstant", 4880.669677734375D,
|
||||
"uniques", QueryRunnerTestHelper.UNIQUES_2
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1592,7 +1849,7 @@ public class TopNQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1621,6 +1878,51 @@ public class TopNQueryRunnerTest
|
|||
)
|
||||
)
|
||||
);
|
||||
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTopNQueryByComplexMetric()
|
||||
{
|
||||
TopNQuery query =
|
||||
new TopNQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.dimension(QueryRunnerTestHelper.marketDimension)
|
||||
.metric(new NumericTopNMetricSpec("numVals"))
|
||||
.threshold(10)
|
||||
.intervals(QueryRunnerTestHelper.firstToThird)
|
||||
.aggregators(
|
||||
Lists.<AggregatorFactory>newArrayList(
|
||||
new CardinalityAggregatorFactory(
|
||||
"numVals", ImmutableList.of(QueryRunnerTestHelper.marketDimension), false
|
||||
)
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
||||
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"market", "spot",
|
||||
"numVals", 1.0002442201269182d
|
||||
),
|
||||
ImmutableMap.<String, Object>of(
|
||||
"market", "total_market",
|
||||
"numVals", 1.0002442201269182d
|
||||
),
|
||||
ImmutableMap.<String, Object>of(
|
||||
"market", "upfront",
|
||||
"numVals", 1.0002442201269182d
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||
}
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
package io.druid.segment;
|
||||
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
public class NullDimensionSelectorTest {
|
||||
|
||||
private final NullDimensionSelector selector = new NullDimensionSelector();
|
||||
|
||||
@Test
|
||||
public void testGetRow() throws Exception {
|
||||
IndexedInts row = selector.getRow();
|
||||
Assert.assertEquals(1, row.size());
|
||||
Assert.assertEquals(0, row.get(0));
|
||||
|
||||
Iterator<Integer> iter = row.iterator();
|
||||
Assert.assertEquals(true, iter.hasNext());
|
||||
Assert.assertEquals(0, iter.next().intValue());
|
||||
Assert.assertEquals(false, iter.hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueCardinality() throws Exception {
|
||||
Assert.assertEquals(1, selector.getValueCardinality());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLookupName() throws Exception {
|
||||
Assert.assertEquals(null, selector.lookupName(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLookupId() throws Exception {
|
||||
Assert.assertEquals(0, selector.lookupId(null));
|
||||
Assert.assertEquals(0, selector.lookupId(""));
|
||||
Assert.assertEquals(-1, selector.lookupId("billy"));
|
||||
}
|
||||
}
|
|
@ -55,6 +55,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -94,7 +95,7 @@ public class SchemalessTestFull
|
|||
public void testCompleteIntersectingSchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -110,7 +111,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -126,7 +127,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -154,7 +155,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -173,7 +174,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -187,7 +188,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -199,7 +200,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -229,7 +230,7 @@ public class SchemalessTestFull
|
|||
public void testEmptyStrings()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -245,7 +246,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -261,7 +262,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -289,7 +290,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -317,7 +318,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -327,7 +328,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -337,7 +338,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -368,7 +369,7 @@ public class SchemalessTestFull
|
|||
public void testNonIntersectingSchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -384,7 +385,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -400,7 +401,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -429,7 +430,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -448,7 +449,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -460,7 +461,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -471,7 +472,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -501,7 +502,7 @@ public class SchemalessTestFull
|
|||
public void testPartialIntersectingSchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -517,7 +518,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -533,7 +534,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -561,7 +562,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -580,7 +581,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -592,7 +593,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -603,7 +604,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -633,7 +634,7 @@ public class SchemalessTestFull
|
|||
public void testSupersetSchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -651,7 +652,7 @@ public class SchemalessTestFull
|
|||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = expectedTimeseriesResults;
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -672,7 +673,7 @@ public class SchemalessTestFull
|
|||
List<Result<TopNResultValue>> expectedFilteredTopNResults = expectedTopNResults;
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -686,7 +687,7 @@ public class SchemalessTestFull
|
|||
List<Result<SearchResultValue>> expectedFilteredSearchResults = expectedSearchResults;
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -716,7 +717,7 @@ public class SchemalessTestFull
|
|||
public void testValueAndEmptySchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -732,7 +733,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -748,7 +749,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -777,7 +778,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -796,7 +797,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -810,7 +811,7 @@ public class SchemalessTestFull
|
|||
List<Result<SearchResultValue>> expectedFilteredSearchResults = expectedSearchResults;
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -840,7 +841,7 @@ public class SchemalessTestFull
|
|||
public void testEmptySchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -856,7 +857,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -871,11 +872,36 @@ public class SchemalessTestFull
|
|||
)
|
||||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList();
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList();
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
new LinkedHashMap<String, Object>(){{
|
||||
put("market", null);
|
||||
put("rows", 1L);
|
||||
put("index", 0.0D);
|
||||
put("addRowsIndexConstant", 2.0D);
|
||||
put("uniques", 0.0D);
|
||||
put("maxIndex", 0.0);
|
||||
put("minIndex", 0.0);
|
||||
|
||||
}}
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<SearchHit>asList()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList()
|
||||
|
@ -885,7 +911,7 @@ public class SchemalessTestFull
|
|||
List<Result<SearchResultValue>> expectedFilteredSearchResults = expectedSearchResults;
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -917,7 +943,7 @@ public class SchemalessTestFull
|
|||
public void testExactSameSchemas()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -935,7 +961,7 @@ public class SchemalessTestFull
|
|||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = expectedTimeseriesResults;
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -956,7 +982,7 @@ public class SchemalessTestFull
|
|||
List<Result<TopNResultValue>> expectedFilteredTopNResults = expectedTopNResults;
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -970,7 +996,7 @@ public class SchemalessTestFull
|
|||
List<Result<SearchResultValue>> expectedFilteredSearchResults = expectedSearchResults;
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -1002,7 +1028,7 @@ public class SchemalessTestFull
|
|||
public void testMultiDimensionalValues()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -1018,7 +1044,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -1034,7 +1060,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1063,7 +1089,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1083,7 +1109,7 @@ public class SchemalessTestFull
|
|||
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -1094,7 +1120,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList()
|
||||
|
@ -1103,7 +1129,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -1136,7 +1162,7 @@ public class SchemalessTestFull
|
|||
public void testDifferentMetrics()
|
||||
{
|
||||
List<Result<TimeseriesResultValue>> expectedTimeseriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -1152,7 +1178,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> expectedFilteredTimeSeriesResults = Arrays.asList(
|
||||
new Result<TimeseriesResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -1206,7 +1232,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
*/
|
||||
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1243,7 +1269,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
|
||||
new Result<TopNResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
|
@ -1271,7 +1297,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -1285,7 +1311,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<SearchResultValue>> expectedFilteredSearchResults = Arrays.asList(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new SearchResultValue(
|
||||
Arrays.<SearchHit>asList(
|
||||
|
@ -1297,7 +1323,7 @@ public class SchemalessTestFull
|
|||
);
|
||||
|
||||
List<Result<TimeBoundaryResultValue>> expectedTimeBoundaryResults = Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
new Result<>(
|
||||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
|
@ -1326,19 +1352,19 @@ public class SchemalessTestFull
|
|||
private List<Pair<QueryableIndex, String>> getIndexes(int index1, int index2)
|
||||
{
|
||||
return Arrays.asList(
|
||||
new Pair<QueryableIndex, String>(
|
||||
new Pair<>(
|
||||
SchemalessIndex.getIncrementalIndex(index1, index2),
|
||||
String.format("Failed: II[%,d, %,d]", index1, index2)
|
||||
),
|
||||
new Pair<QueryableIndex, String>(
|
||||
new Pair<>(
|
||||
SchemalessIndex.getIncrementalIndex(index2, index1),
|
||||
String.format("Failed: II[%,d, %,d]", index2, index1)
|
||||
),
|
||||
new Pair<QueryableIndex, String>(
|
||||
new Pair<>(
|
||||
SchemalessIndex.getMergedIncrementalIndex(index1, index2),
|
||||
String.format("Failed: MII[%,d, %,d]", index1, index2)
|
||||
),
|
||||
new Pair<QueryableIndex, String>(
|
||||
new Pair<>(
|
||||
SchemalessIndex.getMergedIncrementalIndex(index2, index1),
|
||||
String.format("Failed: MII[%,d, %,d]", index2, index1)
|
||||
)
|
||||
|
@ -1422,7 +1448,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " timeseries ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
|
@ -1456,7 +1482,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered timeseries ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
|
@ -1489,7 +1515,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " topN ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TopNResultValue>>newArrayList()
|
||||
|
@ -1523,7 +1549,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered topN ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TopNResultValue>>newArrayList()
|
||||
|
@ -1541,7 +1567,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " search ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<SearchResultValue>>newArrayList()
|
||||
|
@ -1560,7 +1586,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered search ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<SearchResultValue>>newArrayList()
|
||||
|
@ -1579,7 +1605,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " timeBoundary ";
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<>();
|
||||
Iterable<Result<TimeBoundaryResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||
|
|
|
@ -44,4 +44,13 @@ public class ServerConfig
|
|||
{
|
||||
return maxIdleTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ServerConfig{" +
|
||||
"numThreads=" + numThreads +
|
||||
", maxIdleTime=" + maxIdleTime +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,8 +20,7 @@ package io.druid.server.metrics;
|
|||
import com.metamx.emitter.core.Event;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
|
||||
public class
|
||||
NoopServiceEmitter extends ServiceEmitter
|
||||
public class NoopServiceEmitter extends ServiceEmitter
|
||||
{
|
||||
public NoopServiceEmitter()
|
||||
{
|
||||
|
|
|
@ -85,7 +85,6 @@ public class RealtimeModule implements Module
|
|||
Jerseys.addResource(binder, QueryResource.class);
|
||||
Jerseys.addResource(binder, ChatHandlerResource.class);
|
||||
LifecycleModule.register(binder, QueryResource.class);
|
||||
|
||||
LifecycleModule.register(binder, Server.class);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue