mirror of https://github.com/apache/druid.git
1) Adjust the StorageAdapters to lowercase names of metrics and dimensions before looking them up.
2) Add some docs to InputRow/Row to indicate that column names passed into the methods are *always* lowercase and that the rows need to act accordingly. (fixes #29, or at least clarifies the behavior...)
This commit is contained in:
parent
aa073e4e73
commit
701cc9562b
|
@ -39,7 +39,7 @@ public class DefaultDimensionSpec implements DimensionSpec
|
||||||
@JsonProperty("outputName") String outputName
|
@JsonProperty("outputName") String outputName
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension.toLowerCase();
|
this.dimension = dimension;
|
||||||
|
|
||||||
// Do null check for legacy backwards compatibility, callers should be setting the value.
|
// Do null check for legacy backwards compatibility, callers should be setting the value.
|
||||||
this.outputName = outputName == null ? dimension : outputName;
|
this.outputName = outputName == null ? dimension : outputName;
|
||||||
|
|
|
@ -42,7 +42,7 @@ public class ExtractionDimensionSpec implements DimensionSpec
|
||||||
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension.toLowerCase();
|
this.dimension = dimension;
|
||||||
this.dimExtractionFn = dimExtractionFn;
|
this.dimExtractionFn = dimExtractionFn;
|
||||||
|
|
||||||
// Do null check for backwards compatibility
|
// Do null check for backwards compatibility
|
||||||
|
|
|
@ -42,7 +42,7 @@ public class ExtractionDimFilter implements DimFilter
|
||||||
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension.toLowerCase();
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
this.dimExtractionFn = dimExtractionFn;
|
this.dimExtractionFn = dimExtractionFn;
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,7 @@ public class SelectorDimFilter implements DimFilter
|
||||||
@JsonProperty("value") String value
|
@JsonProperty("value") String value
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimension = dimension.toLowerCase();
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -69,7 +69,7 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
|
||||||
@Override
|
@Override
|
||||||
public String apply(@Nullable String input)
|
public String apply(@Nullable String input)
|
||||||
{
|
{
|
||||||
return input.toLowerCase();
|
return input;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -45,7 +45,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName.toLowerCase();
|
this.fieldName = fieldName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -57,7 +57,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.script = expression.toLowerCase();
|
this.script = expression;
|
||||||
this.fieldNames = fieldNames;
|
this.fieldNames = fieldNames;
|
||||||
this.combiner = compileScript(script);
|
this.combiner = compileScript(script);
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,7 +45,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName.toLowerCase();
|
this.fieldName = fieldName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -45,7 +45,7 @@ public class MaxAggregatorFactory implements AggregatorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName.toLowerCase();
|
this.fieldName = fieldName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -45,7 +45,7 @@ public class MinAggregatorFactory implements AggregatorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.fieldName = fieldName.toLowerCase();
|
this.fieldName = fieldName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -22,8 +22,25 @@ package com.metamx.druid.input;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* An InputRow is the interface definition of an event being input into the data ingestion layer.
|
||||||
|
*
|
||||||
|
* An InputRow is a Row with a self-describing list of the dimensions available. This list is used to
|
||||||
|
* implement "schema-less" data ingestion that allows the system to add new dimensions as they appear.
|
||||||
|
*
|
||||||
|
* Note, Druid is a case-insensitive system for parts of schema (column names), this has direct implications
|
||||||
|
* for the implementation of InputRows and Rows. The case-insensitiveness is implemented by lowercasing all
|
||||||
|
* schema elements before looking them up, this means that calls to getDimension() and getFloatMetric() will
|
||||||
|
* have all lowercase column names passed in no matter what is returned from getDimensions or passed in as the
|
||||||
|
* fieldName of an AggregatorFactory. Implementations of InputRow and Row should expect to get values back
|
||||||
|
* in all lowercase form (i.e. they should either have already turned everything into lowercase or they
|
||||||
|
* should operate in a case-insensitive manner).
|
||||||
*/
|
*/
|
||||||
public interface InputRow extends Row
|
public interface InputRow extends Row
|
||||||
{
|
{
|
||||||
|
/**
|
||||||
|
* Returns the dimensions that exist in this row.
|
||||||
|
*
|
||||||
|
* @return the dimensions that exist in this row.
|
||||||
|
*/
|
||||||
public List<String> getDimensions();
|
public List<String> getDimensions();
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,16 @@ import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* A Row of data. This can be used for both input and output into various parts of the system. It assumes
|
||||||
|
* that the user already knows the schema of the row and can query for the parts that they care about.
|
||||||
|
*
|
||||||
|
* Note, Druid is a case-insensitive system for parts of schema (column names), this has direct implications
|
||||||
|
* for the implementation of InputRows and Rows. The case-insensitiveness is implemented by lowercasing all
|
||||||
|
* schema elements before looking them up, this means that calls to getDimension() and getFloatMetric() will
|
||||||
|
* have all lowercase column names passed in no matter what is returned from getDimensions or passed in as the
|
||||||
|
* fieldName of an AggregatorFactory. Implementations of InputRow and Row should expect to get values back
|
||||||
|
* in all lowercase form (i.e. they should either have already turned everything into lowercase or they
|
||||||
|
* should operate in a case-insensitive manner).
|
||||||
*/
|
*/
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "version")
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "version")
|
||||||
@JsonSubTypes(value = {
|
@JsonSubTypes(value = {
|
||||||
|
@ -32,7 +42,31 @@ import java.util.List;
|
||||||
})
|
})
|
||||||
public interface Row
|
public interface Row
|
||||||
{
|
{
|
||||||
|
/**
|
||||||
|
* Returns the timestamp from the epoch in milliseconds. If the event happened _right now_, this would return the
|
||||||
|
* same thing as System.currentTimeMillis();
|
||||||
|
*
|
||||||
|
* @return the timestamp from the epoch in milliseconds.
|
||||||
|
*/
|
||||||
public long getTimestampFromEpoch();
|
public long getTimestampFromEpoch();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the list of dimension values for the given column name.
|
||||||
|
*
|
||||||
|
* Column names are always all lowercase in order to support case-insensitive schemas.
|
||||||
|
*
|
||||||
|
* @param dimension the lowercase column name of the dimension requested
|
||||||
|
* @return the list of values for the provided column name
|
||||||
|
*/
|
||||||
public List<String> getDimension(String dimension);
|
public List<String> getDimension(String dimension);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the float value of the given metric column.
|
||||||
|
*
|
||||||
|
* Column names are always all lowercase in order to support case-insensitive schemas.
|
||||||
|
*
|
||||||
|
* @param metric the lowercase column name of the metric requested
|
||||||
|
* @return the float value for the provided column name.
|
||||||
|
*/
|
||||||
public float getFloatMetric(String metric);
|
public float getFloatMetric(String metric);
|
||||||
}
|
}
|
||||||
|
|
|
@ -162,8 +162,8 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
||||||
private final Runnable commitRunnable = new Runnable() { public void run() {} };
|
private final Runnable commitRunnable = new Runnable() { public void run() {} };
|
||||||
private final java.util.Random rand = (seed == 0L) ? new Random() : new Random(seed);
|
private final java.util.Random rand = (seed == 0L) ? new Random() : new Random(seed);
|
||||||
private final LinkedList<String> dimensions = new LinkedList<String>();
|
private final LinkedList<String> dimensions = new LinkedList<String>();
|
||||||
private final boolean placeholderForAdd = dimensions.add("inColumn".toLowerCase());
|
private final boolean placeholderForAdd = dimensions.add("inColumn");
|
||||||
private final boolean placeholderForAdd2 = dimensions.add("target".toLowerCase());
|
private final boolean placeholderForAdd2 = dimensions.add("target");
|
||||||
|
|
||||||
private final Map<String, Object> theMap = new HashMap<String, Object>(2);
|
private final Map<String, Object> theMap = new HashMap<String, Object>(2);
|
||||||
private long rowCount = 0L;
|
private long rowCount = 0L;
|
||||||
|
@ -200,7 +200,7 @@ public class RandomFirehoseFactory implements FirehoseFactory
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
rowCount++;
|
rowCount++;
|
||||||
theMap.put("inColumn".toLowerCase(), anotherRand((int)nth));
|
theMap.put("inColumn", anotherRand((int)nth));
|
||||||
theMap.put("target", ("a" + nth));
|
theMap.put("target", ("a" + nth));
|
||||||
return new MapBasedInputRow(System.currentTimeMillis(), dimensions, theMap);
|
return new MapBasedInputRow(System.currentTimeMillis(), dimensions, theMap);
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,4 +79,19 @@
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<artifactId>maven-jar-plugin</artifactId>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<goals>
|
||||||
|
<goal>test-jar</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</build>
|
||||||
</project>
|
</project>
|
|
@ -59,6 +59,11 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
||||||
this.baseLongBuffers = baseLongBuffers;
|
this.baseLongBuffers = baseLongBuffers;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IndexedLongs get()
|
public IndexedLongs get()
|
||||||
{
|
{
|
||||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid.index.v1;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.BiMap;
|
import com.google.common.collect.BiMap;
|
||||||
import com.google.common.collect.HashBiMap;
|
import com.google.common.collect.HashBiMap;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -52,6 +53,7 @@ import java.util.Iterator;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.NavigableSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentNavigableMap;
|
import java.util.concurrent.ConcurrentNavigableMap;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
|
@ -65,18 +67,19 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
|
|
||||||
private final long minTimestamp;
|
private final long minTimestamp;
|
||||||
private final QueryGranularity gran;
|
private final QueryGranularity gran;
|
||||||
final AggregatorFactory[] metrics;
|
private final AggregatorFactory[] metrics;
|
||||||
|
|
||||||
private final Map<String, Integer> metricIndexes;
|
private final Map<String, Integer> metricIndexes;
|
||||||
private final Map<String, String> metricTypes;
|
private final Map<String, String> metricTypes;
|
||||||
final LinkedHashMap<String, Integer> dimensionOrder;
|
private final ImmutableList<String> metricNames;
|
||||||
final CopyOnWriteArrayList<String> dimensions;
|
private final LinkedHashMap<String, Integer> dimensionOrder;
|
||||||
final DimensionHolder dimValues;
|
private final CopyOnWriteArrayList<String> dimensions;
|
||||||
final ConcurrentSkipListMap<TimeAndDims, Aggregator[]> facts;
|
private final DimensionHolder dimValues;
|
||||||
|
private final ConcurrentSkipListMap<TimeAndDims, Aggregator[]> facts;
|
||||||
|
|
||||||
private volatile int numEntries = 0;
|
private volatile int numEntries = 0;
|
||||||
|
|
||||||
// This is modified by the same thread.
|
// This is modified on add() by a (hopefully) single thread.
|
||||||
private InputRow in;
|
private InputRow in;
|
||||||
|
|
||||||
public IncrementalIndex(
|
public IncrementalIndex(
|
||||||
|
@ -89,12 +92,16 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
this.gran = gran;
|
this.gran = gran;
|
||||||
this.metrics = metrics;
|
this.metrics = metrics;
|
||||||
|
|
||||||
|
final ImmutableList.Builder<String> metricNamesBuilder = ImmutableList.builder();
|
||||||
final ImmutableMap.Builder<String, Integer> metricIndexesBuilder = ImmutableMap.builder();
|
final ImmutableMap.Builder<String, Integer> metricIndexesBuilder = ImmutableMap.builder();
|
||||||
final ImmutableMap.Builder<String, String> metricTypesBuilder = ImmutableMap.builder();
|
final ImmutableMap.Builder<String, String> metricTypesBuilder = ImmutableMap.builder();
|
||||||
for (int i = 0; i < metrics.length; i++) {
|
for (int i = 0; i < metrics.length; i++) {
|
||||||
metricIndexesBuilder.put(metrics[i].getName().toLowerCase(), i);
|
final String metricName = metrics[i].getName().toLowerCase();
|
||||||
metricTypesBuilder.put(metrics[i].getName().toLowerCase(), metrics[i].getTypeName());
|
metricNamesBuilder.add(metricName);
|
||||||
|
metricIndexesBuilder.put(metricName, i);
|
||||||
|
metricTypesBuilder.put(metricName, metrics[i].getTypeName());
|
||||||
}
|
}
|
||||||
|
metricNames = metricNamesBuilder.build();
|
||||||
metricIndexes = metricIndexesBuilder.build();
|
metricIndexes = metricIndexesBuilder.build();
|
||||||
metricTypes = metricTypesBuilder.build();
|
metricTypes = metricTypesBuilder.build();
|
||||||
|
|
||||||
|
@ -109,7 +116,7 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
||||||
* update that row instead of inserting a new one.
|
* update that row instead of inserting a new one.
|
||||||
* <p/>
|
* <p/>
|
||||||
* This is *not* thread-safe. Calls to add() must be serialized externally
|
* This is *not* thread-safe. Calls to add() should always happen on the same thread.
|
||||||
*
|
*
|
||||||
* @param row the row of data to add
|
* @param row the row of data to add
|
||||||
*
|
*
|
||||||
|
@ -126,6 +133,8 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
|
|
||||||
List<String[]> overflow = null;
|
List<String[]> overflow = null;
|
||||||
for (String dimension : rowDimensions) {
|
for (String dimension : rowDimensions) {
|
||||||
|
dimension = dimension.toLowerCase();
|
||||||
|
|
||||||
final Integer index = dimensionOrder.get(dimension);
|
final Integer index = dimensionOrder.get(dimension);
|
||||||
if (index == null) {
|
if (index == null) {
|
||||||
dimensionOrder.put(dimension, dimensionOrder.size());
|
dimensionOrder.put(dimension, dimensionOrder.size());
|
||||||
|
@ -134,9 +143,9 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
if (overflow == null) {
|
if (overflow == null) {
|
||||||
overflow = Lists.newArrayList();
|
overflow = Lists.newArrayList();
|
||||||
}
|
}
|
||||||
overflow.add(getDimVals(row, dimension));
|
overflow.add(getDimVals(dimValues.add(dimension), row.getDimension(dimension)));
|
||||||
} else {
|
} else {
|
||||||
dims[index] = getDimVals(row, dimension);
|
dims[index] = getDimVals(dimValues.get(dimension), row.getDimension(dimension));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -163,8 +172,9 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
new MetricSelectorFactory()
|
new MetricSelectorFactory()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public FloatMetricSelector makeFloatMetricSelector(final String metricName)
|
public FloatMetricSelector makeFloatMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
return new FloatMetricSelector()
|
return new FloatMetricSelector()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -176,7 +186,7 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ComplexMetricSelector makeComplexMetricSelector(final String metricName)
|
public ComplexMetricSelector makeComplexMetricSelector(final String metric)
|
||||||
{
|
{
|
||||||
final String typeName = agg.getTypeName();
|
final String typeName = agg.getTypeName();
|
||||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||||
|
@ -186,6 +196,7 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
}
|
}
|
||||||
|
|
||||||
final ComplexMetricExtractor extractor = serde.getExtractor();
|
final ComplexMetricExtractor extractor = serde.getExtractor();
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
|
|
||||||
return new ComplexMetricSelector()
|
return new ComplexMetricSelector()
|
||||||
{
|
{
|
||||||
|
@ -227,20 +238,18 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
return numEntries;
|
return numEntries;
|
||||||
}
|
}
|
||||||
|
|
||||||
private long getMinTimeMillis()
|
public long getMinTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.firstKey().getTimestamp();
|
return facts.firstKey().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
private long getMaxTimeMillis()
|
public long getMaxTimeMillis()
|
||||||
{
|
{
|
||||||
return facts.lastKey().getTimestamp();
|
return facts.lastKey().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
private String[] getDimVals(InputRow row, String dimension)
|
private String[] getDimVals(final DimDim dimLookup, final List<String> dimValues)
|
||||||
{
|
{
|
||||||
final DimDim dimLookup = dimValues.getOrAdd(dimension);
|
|
||||||
final List<String> dimValues = row.getDimension(dimension);
|
|
||||||
final String[] retVal = new String[dimValues.size()];
|
final String[] retVal = new String[dimValues.size()];
|
||||||
|
|
||||||
int count = 0;
|
int count = 0;
|
||||||
|
@ -309,11 +318,21 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
return dimensionOrder.get(dimension);
|
return dimensionOrder.get(dimension);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
List<String> getMetricNames()
|
||||||
|
{
|
||||||
|
return metricNames;
|
||||||
|
}
|
||||||
|
|
||||||
Integer getMetricIndex(String metricName)
|
Integer getMetricIndex(String metricName)
|
||||||
{
|
{
|
||||||
return metricIndexes.get(metricName);
|
return metricIndexes.get(metricName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ConcurrentSkipListMap<TimeAndDims, Aggregator[]> getFacts()
|
||||||
|
{
|
||||||
|
return facts;
|
||||||
|
}
|
||||||
|
|
||||||
ConcurrentNavigableMap<TimeAndDims, Aggregator[]> getSubMap(TimeAndDims start, TimeAndDims end)
|
ConcurrentNavigableMap<TimeAndDims, Aggregator[]> getSubMap(TimeAndDims start, TimeAndDims end)
|
||||||
{
|
{
|
||||||
return facts.subMap(start, end);
|
return facts.subMap(start, end);
|
||||||
|
@ -366,13 +385,16 @@ public class IncrementalIndex implements Iterable<Row>
|
||||||
dimensions.clear();
|
dimensions.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
DimDim getOrAdd(String dimension)
|
DimDim add(String dimension)
|
||||||
{
|
{
|
||||||
DimDim holder = dimensions.get(dimension);
|
DimDim holder = dimensions.get(dimension);
|
||||||
if (holder == null) {
|
if (holder == null) {
|
||||||
holder = new DimDim();
|
holder = new DimDim();
|
||||||
dimensions.put(dimension, holder);
|
dimensions.put(dimension, holder);
|
||||||
}
|
}
|
||||||
|
else {
|
||||||
|
throw new ISE("dimension[%s] already existed even though add() was called!?", dimension);
|
||||||
|
}
|
||||||
return holder;
|
return holder;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,17 +19,12 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexer.data;
|
package com.metamx.druid.indexer.data;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.base.Joiner;
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.metamx.common.parsers.CSVParser;
|
import com.metamx.common.parsers.CSVParser;
|
||||||
import com.metamx.common.parsers.Parser;
|
import com.metamx.common.parsers.Parser;
|
||||||
import com.metamx.common.parsers.ToLowerCaseParser;
|
|
||||||
import org.codehaus.jackson.annotate.JsonCreator;
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
import org.codehaus.jackson.annotate.JsonProperty;
|
import org.codehaus.jackson.annotate.JsonProperty;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -45,33 +40,13 @@ public class CSVDataSpec implements DataSpec
|
||||||
@JsonProperty("dimensions") List<String> dimensions
|
@JsonProperty("dimensions") List<String> dimensions
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(columns);
|
Preconditions.checkNotNull(columns, "columns");
|
||||||
Preconditions.checkArgument(
|
for (String column : columns) {
|
||||||
!Joiner.on("_").join(columns).contains(","), "Columns must not have commas in them"
|
Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
|
||||||
);
|
}
|
||||||
|
|
||||||
this.columns = Lists.transform(
|
this.columns = columns;
|
||||||
columns,
|
this.dimensions = dimensions;
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
this.dimensions = (dimensions == null) ? dimensions : Lists.transform(
|
|
||||||
dimensions,
|
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("columns")
|
@JsonProperty("columns")
|
||||||
|
@ -102,8 +77,8 @@ public class CSVDataSpec implements DataSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Parser getParser()
|
public Parser<String, Object> getParser()
|
||||||
{
|
{
|
||||||
return new ToLowerCaseParser(new CSVParser(columns));
|
return new CSVParser(columns);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,17 +19,12 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexer.data;
|
package com.metamx.druid.indexer.data;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.base.Joiner;
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.metamx.common.parsers.DelimitedParser;
|
import com.metamx.common.parsers.DelimitedParser;
|
||||||
import com.metamx.common.parsers.Parser;
|
import com.metamx.common.parsers.Parser;
|
||||||
import com.metamx.common.parsers.ToLowerCaseParser;
|
|
||||||
import org.codehaus.jackson.annotate.JsonCreator;
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
import org.codehaus.jackson.annotate.JsonProperty;
|
import org.codehaus.jackson.annotate.JsonProperty;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -48,33 +43,13 @@ public class DelimitedDataSpec implements DataSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(columns);
|
Preconditions.checkNotNull(columns);
|
||||||
Preconditions.checkArgument(
|
for (String column : columns) {
|
||||||
!Joiner.on("_").join(columns).contains(","), "Columns must not have commas in them"
|
Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
|
||||||
);
|
}
|
||||||
|
|
||||||
this.delimiter = (delimiter == null) ? DelimitedParser.DEFAULT_DELIMITER : delimiter;
|
this.delimiter = (delimiter == null) ? DelimitedParser.DEFAULT_DELIMITER : delimiter;
|
||||||
this.columns = Lists.transform(
|
this.columns = columns;
|
||||||
columns,
|
this.dimensions = dimensions;
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
this.dimensions = (dimensions == null) ? dimensions : Lists.transform(
|
|
||||||
dimensions,
|
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("delimiter")
|
@JsonProperty("delimiter")
|
||||||
|
@ -111,10 +86,10 @@ public class DelimitedDataSpec implements DataSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Parser getParser()
|
public Parser<String, Object> getParser()
|
||||||
{
|
{
|
||||||
Parser retVal = new DelimitedParser(delimiter);
|
Parser<String, Object> retVal = new DelimitedParser(delimiter);
|
||||||
retVal.setFieldNames(columns);
|
retVal.setFieldNames(columns);
|
||||||
return new ToLowerCaseParser(retVal);
|
return retVal;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,14 +19,10 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexer.data;
|
package com.metamx.druid.indexer.data;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.metamx.common.parsers.JSONParser;
|
import com.metamx.common.parsers.JSONParser;
|
||||||
import com.metamx.common.parsers.Parser;
|
import com.metamx.common.parsers.Parser;
|
||||||
import com.metamx.common.parsers.ToLowerCaseParser;
|
|
||||||
import org.codehaus.jackson.annotate.JsonProperty;
|
import org.codehaus.jackson.annotate.JsonProperty;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -39,17 +35,7 @@ public class JSONDataSpec implements DataSpec
|
||||||
@JsonProperty("dimensions") List<String> dimensions
|
@JsonProperty("dimensions") List<String> dimensions
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dimensions = (dimensions == null) ? dimensions : Lists.transform(
|
this.dimensions = dimensions;
|
||||||
dimensions,
|
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("dimensions")
|
@JsonProperty("dimensions")
|
||||||
|
@ -71,8 +57,8 @@ public class JSONDataSpec implements DataSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Parser getParser()
|
public Parser<String, Object> getParser()
|
||||||
{
|
{
|
||||||
return new ToLowerCaseParser(new JSONParser());
|
return new JSONParser();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.exception.FormattedException;
|
import com.metamx.common.exception.FormattedException;
|
||||||
import com.metamx.common.parsers.Parser;
|
import com.metamx.common.parsers.Parser;
|
||||||
|
import com.metamx.common.parsers.ToLowerCaseParser;
|
||||||
import com.metamx.druid.input.InputRow;
|
import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.input.MapBasedInputRow;
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
import org.codehaus.jackson.annotate.JsonCreator;
|
import org.codehaus.jackson.annotate.JsonCreator;
|
||||||
|
@ -59,7 +60,7 @@ public class StringInputRowParser
|
||||||
}
|
}
|
||||||
this.dimensionExclusions.add(timestampSpec.getTimestampColumn());
|
this.dimensionExclusions.add(timestampSpec.getTimestampColumn());
|
||||||
|
|
||||||
this.parser = dataSpec.getParser();
|
this.parser = new ToLowerCaseParser(dataSpec.getParser());
|
||||||
}
|
}
|
||||||
|
|
||||||
public StringInputRowParser addDimensionExclusion(String dimension)
|
public StringInputRowParser addDimensionExclusion(String dimension)
|
||||||
|
|
|
@ -0,0 +1,70 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.indexer.data;
|
||||||
|
|
||||||
|
import com.metamx.common.parsers.Parser;
|
||||||
|
import com.metamx.common.parsers.ToLowerCaseParser;
|
||||||
|
import org.codehaus.jackson.annotate.JsonValue;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class ToLowercaseDataSpec implements DataSpec
|
||||||
|
{
|
||||||
|
private final DataSpec delegate;
|
||||||
|
|
||||||
|
public ToLowercaseDataSpec(
|
||||||
|
DataSpec delegate
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verify(List<String> usedCols)
|
||||||
|
{
|
||||||
|
delegate.verify(usedCols);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasCustomDimensions()
|
||||||
|
{
|
||||||
|
return delegate.hasCustomDimensions();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> getDimensions()
|
||||||
|
{
|
||||||
|
return delegate.getDimensions();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Parser<String, Object> getParser()
|
||||||
|
{
|
||||||
|
return new ToLowerCaseParser(delegate.getParser());
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonValue
|
||||||
|
public DataSpec getDelegate()
|
||||||
|
{
|
||||||
|
return delegate;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,81 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.index.v1;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
|
import com.metamx.druid.input.Row;
|
||||||
|
import junit.framework.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class IncrementalIndexTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCaseInsensitivity() throws Exception
|
||||||
|
{
|
||||||
|
final long timestamp = System.currentTimeMillis();
|
||||||
|
|
||||||
|
IncrementalIndex index = createCaseInsensitiveIndex(timestamp);
|
||||||
|
|
||||||
|
Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensions());
|
||||||
|
Assert.assertEquals(2, index.size());
|
||||||
|
|
||||||
|
final Iterator<Row> rows = index.iterator();
|
||||||
|
Row row = rows.next();
|
||||||
|
Assert.assertEquals(timestamp, row.getTimestampFromEpoch());
|
||||||
|
Assert.assertEquals(Arrays.asList("1"), row.getDimension("dim1"));
|
||||||
|
Assert.assertEquals(Arrays.asList("2"), row.getDimension("dim2"));
|
||||||
|
|
||||||
|
row = rows.next();
|
||||||
|
Assert.assertEquals(timestamp, row.getTimestampFromEpoch());
|
||||||
|
Assert.assertEquals(Arrays.asList("3"), row.getDimension("dim1"));
|
||||||
|
Assert.assertEquals(Arrays.asList("4"), row.getDimension("dim2"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static IncrementalIndex createCaseInsensitiveIndex(long timestamp)
|
||||||
|
{
|
||||||
|
IncrementalIndex index = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||||
|
|
||||||
|
index.add(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
timestamp,
|
||||||
|
Arrays.asList("Dim1", "DiM2"),
|
||||||
|
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2", "DIM1", "3", "dIM2", "4")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
index.add(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
timestamp,
|
||||||
|
Arrays.asList("diM1", "dIM2"),
|
||||||
|
ImmutableMap.<String, Object>of("Dim1", "1", "DiM2", "2", "dim1", "3", "dim2", "4")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
return index;
|
||||||
|
}
|
||||||
|
}
|
|
@ -36,6 +36,7 @@ import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.RegisteringNode;
|
import com.metamx.druid.RegisteringNode;
|
||||||
import com.metamx.druid.index.v1.serde.Registererer;
|
import com.metamx.druid.index.v1.serde.Registererer;
|
||||||
import com.metamx.druid.indexer.data.DataSpec;
|
import com.metamx.druid.indexer.data.DataSpec;
|
||||||
|
import com.metamx.druid.indexer.data.ToLowercaseDataSpec;
|
||||||
import com.metamx.druid.indexer.granularity.GranularitySpec;
|
import com.metamx.druid.indexer.granularity.GranularitySpec;
|
||||||
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
||||||
import com.metamx.druid.indexer.path.PathSpec;
|
import com.metamx.druid.indexer.path.PathSpec;
|
||||||
|
@ -219,7 +220,7 @@ public class HadoopDruidIndexerConfig
|
||||||
|
|
||||||
public void setTimestampColumnName(String timestampColumnName)
|
public void setTimestampColumnName(String timestampColumnName)
|
||||||
{
|
{
|
||||||
this.timestampColumnName = timestampColumnName.toLowerCase();
|
this.timestampColumnName = timestampColumnName;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty()
|
@JsonProperty()
|
||||||
|
@ -241,7 +242,7 @@ public class HadoopDruidIndexerConfig
|
||||||
|
|
||||||
public void setDataSpec(DataSpec dataSpec)
|
public void setDataSpec(DataSpec dataSpec)
|
||||||
{
|
{
|
||||||
this.dataSpec = dataSpec;
|
this.dataSpec = new ToLowercaseDataSpec(dataSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
|
@ -326,7 +327,7 @@ public class HadoopDruidIndexerConfig
|
||||||
|
|
||||||
public void setPartitionDimension(String partitionDimension)
|
public void setPartitionDimension(String partitionDimension)
|
||||||
{
|
{
|
||||||
this.partitionDimension = (partitionDimension == null) ? partitionDimension : partitionDimension.toLowerCase();
|
this.partitionDimension = (partitionDimension == null) ? partitionDimension : partitionDimension;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean partitionByDimension()
|
public boolean partitionByDimension()
|
||||||
|
|
|
@ -76,12 +76,7 @@ public class DeleteTask extends AbstractTask
|
||||||
{
|
{
|
||||||
// Strategy: Create an empty segment covering the interval to be deleted
|
// Strategy: Create an empty segment covering the interval to be deleted
|
||||||
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(
|
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(this.getInterval(), empty);
|
||||||
this.getInterval(),
|
|
||||||
empty,
|
|
||||||
new ArrayList<String>(),
|
|
||||||
new ArrayList<String>()
|
|
||||||
);
|
|
||||||
|
|
||||||
// Create DataSegment
|
// Create DataSegment
|
||||||
final DataSegment segment =
|
final DataSegment segment =
|
||||||
|
|
|
@ -106,7 +106,7 @@ public class TaskConsumer implements Runnable
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to hand off task")
|
log.makeAlert(e, "Failed to hand off task")
|
||||||
.addData("task", task.getId())
|
.addData("task", task.getId())
|
||||||
.addData("type", task.getType().toString().toLowerCase())
|
.addData("type", task.getType().toString())
|
||||||
.addData("dataSource", task.getDataSource())
|
.addData("dataSource", task.getDataSource())
|
||||||
.addData("interval", task.getInterval())
|
.addData("interval", task.getInterval())
|
||||||
.emit();
|
.emit();
|
||||||
|
@ -136,7 +136,7 @@ public class TaskConsumer implements Runnable
|
||||||
);
|
);
|
||||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
||||||
.setUser2(task.getDataSource())
|
.setUser2(task.getDataSource())
|
||||||
.setUser4(task.getType().toString().toLowerCase())
|
.setUser4(task.getType().toString())
|
||||||
.setUser5(task.getInterval().toString());
|
.setUser5(task.getInterval().toString());
|
||||||
|
|
||||||
// Run preflight checks
|
// Run preflight checks
|
||||||
|
@ -233,7 +233,7 @@ public class TaskConsumer implements Runnable
|
||||||
bytes += segment.getSize();
|
bytes += segment.getSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
builder.setUser3(status.getStatusCode().toString().toLowerCase());
|
builder.setUser3(status.getStatusCode().toString());
|
||||||
|
|
||||||
emitter.emit(builder.build("indexer/time/run/millis", status.getDuration()));
|
emitter.emit(builder.build("indexer/time/run/millis", status.getDuration()));
|
||||||
emitter.emit(builder.build("indexer/segment/count", status.getSegments().size()));
|
emitter.emit(builder.build("indexer/segment/count", status.getSegments().size()));
|
||||||
|
@ -245,7 +245,7 @@ public class TaskConsumer implements Runnable
|
||||||
String.format("Failed to index: %s", task.getDataSource()),
|
String.format("Failed to index: %s", task.getDataSource()),
|
||||||
ImmutableMap.<String, Object>builder()
|
ImmutableMap.<String, Object>builder()
|
||||||
.put("task", task.getId())
|
.put("task", task.getId())
|
||||||
.put("type", task.getType().toString().toLowerCase())
|
.put("type", task.getType().toString())
|
||||||
.put("dataSource", task.getDataSource())
|
.put("dataSource", task.getDataSource())
|
||||||
.put("interval", task.getInterval())
|
.put("interval", task.getInterval())
|
||||||
.build()
|
.build()
|
||||||
|
|
6
pom.xml
6
pom.xml
|
@ -41,11 +41,11 @@
|
||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<modules>
|
<modules>
|
||||||
<module>server</module>
|
|
||||||
<module>client</module>
|
|
||||||
<module>common</module>
|
<module>common</module>
|
||||||
<module>indexer</module>
|
|
||||||
<module>index-common</module>
|
<module>index-common</module>
|
||||||
|
<module>client</module>
|
||||||
|
<module>indexer</module>
|
||||||
|
<module>server</module>
|
||||||
<module>merger</module>
|
<module>merger</module>
|
||||||
<module>realtime</module>
|
<module>realtime</module>
|
||||||
<module>examples</module>
|
<module>examples</module>
|
||||||
|
|
|
@ -216,6 +216,13 @@
|
||||||
<type>test-jar</type>
|
<type>test-jar</type>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.metamx.druid</groupId>
|
||||||
|
<artifactId>druid-index-common</artifactId>
|
||||||
|
<version>${project.parent.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
|
|
|
@ -38,7 +38,7 @@ public class AndFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
if (filters.size() == 1) {
|
if (filters.size() == 1) {
|
||||||
return filters.get(0).goConcise(selector);
|
return filters.get(0).goConcise(selector);
|
||||||
|
|
|
@ -19,17 +19,14 @@
|
||||||
|
|
||||||
package com.metamx.druid.index.brita;
|
package com.metamx.druid.index.brita;
|
||||||
|
|
||||||
import com.metamx.druid.index.v1.processing.Offset;
|
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface InvertedIndexSelector
|
public interface BitmapIndexSelector
|
||||||
{
|
{
|
||||||
public Indexed<String> getDimensionValues(String dimension);
|
public Indexed<String> getDimensionValues(String dimension);
|
||||||
public int getNumRows();
|
public int getNumRows();
|
||||||
public int[] getInvertedIndex(String dimension, String value);
|
|
||||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value);
|
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value);
|
||||||
public Offset getInvertedIndexOffset(String dimension, String value);
|
|
||||||
}
|
}
|
|
@ -43,7 +43,7 @@ class DimensionPredicateFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(final InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return ImmutableConciseSet.union(
|
return ImmutableConciseSet.union(
|
||||||
FunctionalIterable.create(selector.getDimensionValues(dimension))
|
FunctionalIterable.create(selector.getDimensionValues(dimension))
|
||||||
|
|
|
@ -47,7 +47,7 @@ public class ExtractionFilter implements Filter
|
||||||
this.fn = fn;
|
this.fn = fn;
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<Filter> makeFilters(InvertedIndexSelector selector)
|
private List<Filter> makeFilters(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
final Indexed<String> allDimVals = selector.getDimensionValues(dimension);
|
final Indexed<String> allDimVals = selector.getDimensionValues(dimension);
|
||||||
final List<Filter> filters = Lists.newArrayList();
|
final List<Filter> filters = Lists.newArrayList();
|
||||||
|
@ -63,7 +63,7 @@ public class ExtractionFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return new OrFilter(makeFilters(selector)).goConcise(selector);
|
return new OrFilter(makeFilters(selector)).goConcise(selector);
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,6 @@ import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||||
*/
|
*/
|
||||||
public interface Filter
|
public interface Filter
|
||||||
{
|
{
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector);
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector);
|
||||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory);
|
public ValueMatcher makeMatcher(ValueMatcherFactory factory);
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,7 +35,7 @@ public class NotFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return ImmutableConciseSet.complement(
|
return ImmutableConciseSet.complement(
|
||||||
baseFilter.goConcise(selector),
|
baseFilter.goConcise(selector),
|
||||||
|
|
|
@ -42,7 +42,7 @@ public class OrFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
if (filters.size() == 1) {
|
if (filters.size() == 1) {
|
||||||
return filters.get(0).goConcise(selector);
|
return filters.get(0).goConcise(selector);
|
||||||
|
|
|
@ -39,7 +39,7 @@ public class SelectorFilter implements Filter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet goConcise(InvertedIndexSelector selector)
|
public ImmutableConciseSet goConcise(BitmapIndexSelector selector)
|
||||||
{
|
{
|
||||||
return selector.getConciseInvertedIndex(dimension, value);
|
return selector.getConciseInvertedIndex(dimension, value);
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Maps;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.aggregation.Aggregator;
|
import com.metamx.druid.aggregation.Aggregator;
|
||||||
|
import com.metamx.druid.input.Row;
|
||||||
import com.metamx.druid.kv.EmptyIndexedInts;
|
import com.metamx.druid.kv.EmptyIndexedInts;
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import com.metamx.druid.kv.IndexedInts;
|
import com.metamx.druid.kv.IndexedInts;
|
||||||
|
@ -46,39 +47,36 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
|
|
||||||
private final Interval dataInterval;
|
private final Interval dataInterval;
|
||||||
private final IncrementalIndex index;
|
private final IncrementalIndex index;
|
||||||
private final List<String> dimensions;
|
|
||||||
private final List<String> metrics;
|
|
||||||
|
|
||||||
private final Map<String, Map<String, ConciseSet>> invertedIndexes;
|
private final Map<String, Map<String, ConciseSet>> invertedIndexes;
|
||||||
|
|
||||||
public IncrementalIndexAdapter(
|
public IncrementalIndexAdapter(
|
||||||
Interval dataInterval, IncrementalIndex index, List<String> dimensions, List<String> metrics
|
Interval dataInterval, IncrementalIndex index
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dataInterval = dataInterval;
|
this.dataInterval = dataInterval;
|
||||||
this.index = index;
|
this.index = index;
|
||||||
this.dimensions = dimensions;
|
|
||||||
this.metrics = metrics;
|
|
||||||
|
|
||||||
this.invertedIndexes = Maps.newHashMap();
|
this.invertedIndexes = Maps.newHashMap();
|
||||||
|
|
||||||
for (String dimension : dimensions) {
|
for (String dimension : index.getDimensions()) {
|
||||||
invertedIndexes.put(dimension, Maps.<String, ConciseSet>newHashMap());
|
invertedIndexes.put(dimension, Maps.<String, ConciseSet>newHashMap());
|
||||||
}
|
}
|
||||||
|
|
||||||
int rowNum = 0;
|
int rowNum = 0;
|
||||||
for (IncrementalIndex.TimeAndDims timeAndDims : index.facts.keySet()) {
|
for (Row row : index) {
|
||||||
|
|
||||||
|
}
|
||||||
|
for (IncrementalIndex.TimeAndDims timeAndDims : index.getFacts().keySet()) {
|
||||||
final String[][] dims = timeAndDims.getDims();
|
final String[][] dims = timeAndDims.getDims();
|
||||||
|
|
||||||
for (String dimension : dimensions) {
|
for (String dimension : index.getDimensions()) {
|
||||||
if (index.dimensionOrder == null || invertedIndexes == null) {
|
int dimIndex = index.getDimensionIndex(dimension);
|
||||||
log.error("wtf, dimensionOrder and indvertedIndexes are null");
|
|
||||||
}
|
|
||||||
int dimIndex = index.dimensionOrder.get(dimension);
|
|
||||||
Map<String, ConciseSet> conciseSets = invertedIndexes.get(dimension);
|
Map<String, ConciseSet> conciseSets = invertedIndexes.get(dimension);
|
||||||
|
|
||||||
if (conciseSets == null || dims == null) {
|
if (conciseSets == null || dims == null) {
|
||||||
log.error("conciseSets and dims are null!");
|
log.error("conciseSets and dims are null!");
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
if (dimIndex >= dims.length || dims[dimIndex] == null) {
|
if (dimIndex >= dims.length || dims[dimIndex] == null) {
|
||||||
continue;
|
continue;
|
||||||
|
@ -120,19 +118,19 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getAvailableDimensions()
|
public Indexed<String> getAvailableDimensions()
|
||||||
{
|
{
|
||||||
return new ListIndexed<String>(dimensions, String.class);
|
return new ListIndexed<String>(index.getDimensions(), String.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getAvailableMetrics()
|
public Indexed<String> getAvailableMetrics()
|
||||||
{
|
{
|
||||||
return new ListIndexed<String>(metrics, String.class);
|
return new ListIndexed<String>(index.getMetricNames(), String.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<String> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
final IncrementalIndex.DimDim dimDim = index.dimValues.get(dimension);
|
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
||||||
dimDim.sort();
|
dimDim.sort();
|
||||||
|
|
||||||
return new Indexed<String>()
|
return new Indexed<String>()
|
||||||
|
@ -173,7 +171,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
public Iterable<Rowboat> getRows()
|
public Iterable<Rowboat> getRows()
|
||||||
{
|
{
|
||||||
return FunctionalIterable
|
return FunctionalIterable
|
||||||
.create(index.facts.entrySet())
|
.create(index.getFacts().entrySet())
|
||||||
.transform(
|
.transform(
|
||||||
new Function<Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]>, Rowboat>()
|
new Function<Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]>, Rowboat>()
|
||||||
{
|
{
|
||||||
|
@ -189,9 +187,9 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
final Aggregator[] aggs = input.getValue();
|
final Aggregator[] aggs = input.getValue();
|
||||||
|
|
||||||
int[][] dims = new int[dimValues.length][];
|
int[][] dims = new int[dimValues.length][];
|
||||||
for (String dimension : dimensions) {
|
for (String dimension : index.getDimensions()) {
|
||||||
int dimIndex = index.dimensionOrder.get(dimension);
|
int dimIndex = index.getDimensionIndex(dimension);
|
||||||
final IncrementalIndex.DimDim dimDim = index.dimValues.get(dimension);
|
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
||||||
dimDim.sort();
|
dimDim.sort();
|
||||||
|
|
||||||
if (dimIndex >= dimValues.length || dimValues[dimIndex] == null) {
|
if (dimIndex >= dimValues.length || dimValues[dimIndex] == null) {
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public int getDimensionCardinality(String dimension)
|
public int getDimensionCardinality(String dimension)
|
||||||
{
|
{
|
||||||
IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
IncrementalIndex.DimDim dimDim = index.getDimension(dimension.toLowerCase());
|
||||||
if (dimDim == null) {
|
if (dimDim == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -232,8 +232,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(String dimensionName)
|
public DimensionSelector makeDimensionSelector(String dimension)
|
||||||
{
|
{
|
||||||
|
final String dimensionName = dimension.toLowerCase();
|
||||||
final IncrementalIndex.DimDim dimValLookup = index.getDimension(dimensionName);
|
final IncrementalIndex.DimDim dimValLookup = index.getDimension(dimensionName);
|
||||||
if (dimValLookup == null) {
|
if (dimValLookup == null) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -303,8 +304,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatMetricSelector makeFloatMetricSelector(String metricName)
|
public FloatMetricSelector makeFloatMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
||||||
if (metricIndexInt == null) {
|
if (metricIndexInt == null) {
|
||||||
return new FloatMetricSelector()
|
return new FloatMetricSelector()
|
||||||
|
@ -330,8 +332,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ComplexMetricSelector makeComplexMetricSelector(String metricName)
|
public ComplexMetricSelector makeComplexMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
||||||
if (metricIndexInt == null) {
|
if (metricIndexInt == null) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -385,7 +388,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
String[] tmpDimensionNames = new String[dimensions.size()];
|
String[] tmpDimensionNames = new String[dimensions.size()];
|
||||||
int i = 0;
|
int i = 0;
|
||||||
for (String dimension : dimensions) {
|
for (String dimension : dimensions) {
|
||||||
Integer dimIndex = index.getDimensionIndex(dimension);
|
Integer dimIndex = index.getDimensionIndex(dimension.toLowerCase());
|
||||||
if (dimIndex != null) {
|
if (dimIndex != null) {
|
||||||
tmpDimensionNames[i] = dimension;
|
tmpDimensionNames[i] = dimension;
|
||||||
tmpDimensionIndexes[i] = dimIndex;
|
tmpDimensionIndexes[i] = dimIndex;
|
||||||
|
@ -495,11 +498,11 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeValueMatcher(String dimension, String value)
|
public ValueMatcher makeValueMatcher(String dimension, String value)
|
||||||
{
|
{
|
||||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase());
|
||||||
if (dimIndexObject == null) {
|
if (dimIndexObject == null) {
|
||||||
return new BooleanValueMatcher(false);
|
return new BooleanValueMatcher(false);
|
||||||
}
|
}
|
||||||
String idObject = index.getDimension(dimension).get(value);
|
String idObject = index.getDimension(dimension.toLowerCase()).get(value);
|
||||||
if (idObject == null) {
|
if (idObject == null) {
|
||||||
return new BooleanValueMatcher(false);
|
return new BooleanValueMatcher(false);
|
||||||
}
|
}
|
||||||
|
@ -530,7 +533,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public ValueMatcher makeValueMatcher(String dimension, final Predicate<String> predicate)
|
public ValueMatcher makeValueMatcher(String dimension, final Predicate<String> predicate)
|
||||||
{
|
{
|
||||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase());
|
||||||
if (dimIndexObject == null) {
|
if (dimIndexObject == null) {
|
||||||
return new BooleanValueMatcher(false);
|
return new BooleanValueMatcher(false);
|
||||||
}
|
}
|
||||||
|
|
|
@ -112,8 +112,8 @@ public class IndexMerger
|
||||||
final IncrementalIndex index, final Interval dataInterval, File outDir, ProgressIndicator progress
|
final IncrementalIndex index, final Interval dataInterval, File outDir, ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final long firstTimestamp = index.facts.firstKey().getTimestamp();
|
final long firstTimestamp = index.getMinTime().getMillis();
|
||||||
final long lastTimestamp = index.facts.lastKey().getTimestamp();
|
final long lastTimestamp = index.getMaxTime().getMillis();
|
||||||
if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) {
|
if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) {
|
||||||
throw new IAE(
|
throw new IAE(
|
||||||
"interval[%s] does not encapsulate the full range of timestamps[%s, %s]",
|
"interval[%s] does not encapsulate the full range of timestamps[%s, %s]",
|
||||||
|
@ -130,26 +130,10 @@ public class IndexMerger
|
||||||
throw new ISE("Can only persist to directories, [%s] wasn't a directory", outDir);
|
throw new ISE("Can only persist to directories, [%s] wasn't a directory", outDir);
|
||||||
}
|
}
|
||||||
|
|
||||||
final List<String> dimensions = Lists.transform(
|
|
||||||
Lists.newArrayList(index.dimensionOrder.keySet()),
|
|
||||||
new Function<String, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable String input)
|
|
||||||
{
|
|
||||||
return input.toLowerCase();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
final List<String> metrics = Lists.newArrayListWithCapacity(index.metrics.length);
|
|
||||||
for (int i = 0; i < index.metrics.length; ++i) {
|
|
||||||
metrics.add(index.metrics[i].getName().toLowerCase());
|
|
||||||
}
|
|
||||||
|
|
||||||
log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
|
log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
|
||||||
return merge(
|
return merge(
|
||||||
Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index, dimensions, metrics)),
|
Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index)),
|
||||||
index.metrics,
|
index.getMetricAggs(),
|
||||||
outDir,
|
outDir,
|
||||||
progress
|
progress
|
||||||
);
|
);
|
||||||
|
|
|
@ -31,8 +31,8 @@ import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.BaseStorageAdapter;
|
import com.metamx.druid.BaseStorageAdapter;
|
||||||
import com.metamx.druid.Capabilities;
|
import com.metamx.druid.Capabilities;
|
||||||
import com.metamx.druid.QueryGranularity;
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.index.brita.BitmapIndexSelector;
|
||||||
import com.metamx.druid.index.brita.Filter;
|
import com.metamx.druid.index.brita.Filter;
|
||||||
import com.metamx.druid.index.brita.InvertedIndexSelector;
|
|
||||||
import com.metamx.druid.index.v1.processing.ArrayBasedOffset;
|
import com.metamx.druid.index.v1.processing.ArrayBasedOffset;
|
||||||
import com.metamx.druid.index.v1.processing.Cursor;
|
import com.metamx.druid.index.v1.processing.Cursor;
|
||||||
import com.metamx.druid.index.v1.processing.DimensionSelector;
|
import com.metamx.druid.index.v1.processing.DimensionSelector;
|
||||||
|
@ -143,7 +143,7 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
baseOffset = new ArrayBasedOffset(ids, intervalStartAndEnd.lhs);
|
baseOffset = new ArrayBasedOffset(ids, intervalStartAndEnd.lhs);
|
||||||
} else {
|
} else {
|
||||||
baseOffset = new StartLimitedOffset(
|
baseOffset = new StartLimitedOffset(
|
||||||
new ConciseOffset(filter.goConcise(new IndexBasedInvertedIndexSelector(index))),
|
new ConciseOffset(filter.goConcise(new IndexBasedBitmapIndexSelector(index))),
|
||||||
intervalStartAndEnd.lhs
|
intervalStartAndEnd.lhs
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -199,8 +199,9 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(final String dimensionName)
|
public DimensionSelector makeDimensionSelector(String dimension)
|
||||||
{
|
{
|
||||||
|
final String dimensionName = dimension.toLowerCase();
|
||||||
final String[] nameLookup = index.reverseDimLookup.get(dimensionName);
|
final String[] nameLookup = index.reverseDimLookup.get(dimensionName);
|
||||||
if (nameLookup == null) {
|
if (nameLookup == null) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -242,9 +243,10 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatMetricSelector makeFloatMetricSelector(String metricName)
|
public FloatMetricSelector makeFloatMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
IndexedFloats cachedFloats = (IndexedFloats) metricHolderCache.get(metricName);
|
String metricName = metric.toLowerCase();
|
||||||
|
IndexedFloats cachedFloats = (IndexedFloats) metricHolderCache.get(metric);
|
||||||
if (cachedFloats == null) {
|
if (cachedFloats == null) {
|
||||||
MetricHolder holder = index.metricVals.get(metricName);
|
MetricHolder holder = index.metricVals.get(metricName);
|
||||||
if (holder == null) {
|
if (holder == null) {
|
||||||
|
@ -274,8 +276,9 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ComplexMetricSelector makeComplexMetricSelector(String metricName)
|
public ComplexMetricSelector makeComplexMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
Indexed cachedComplex = (Indexed) metricHolderCache.get(metricName);
|
Indexed cachedComplex = (Indexed) metricHolderCache.get(metricName);
|
||||||
if (cachedComplex == null) {
|
if (cachedComplex == null) {
|
||||||
MetricHolder holder = index.metricVals.get(metricName);
|
MetricHolder holder = index.metricVals.get(metricName);
|
||||||
|
@ -335,23 +338,21 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<String> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
return new ListIndexed<String>(Lists.newArrayList(index.dimIdLookup.get(dimension).keySet()), String.class);
|
return new ListIndexed<String>(
|
||||||
|
Lists.newArrayList(index.dimIdLookup.get(dimension.toLowerCase()).keySet()), String.class
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal)
|
public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal)
|
||||||
{
|
{
|
||||||
return index.getInvertedIndex(dimension, dimVal);
|
return index.getInvertedIndex(dimension.toLowerCase(), dimVal);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Offset getFilterOffset(Filter filter)
|
public Offset getFilterOffset(Filter filter)
|
||||||
{
|
{
|
||||||
return new ConciseOffset(
|
return new ConciseOffset(filter.goConcise(new IndexBasedBitmapIndexSelector(index)));
|
||||||
filter.goConcise(
|
|
||||||
new IndexBasedInvertedIndexSelector(index)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -459,11 +460,11 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class IndexBasedInvertedIndexSelector implements InvertedIndexSelector
|
private static class IndexBasedBitmapIndexSelector implements BitmapIndexSelector
|
||||||
{
|
{
|
||||||
private final Index index;
|
private final Index index;
|
||||||
|
|
||||||
public IndexBasedInvertedIndexSelector(final Index index)
|
public IndexBasedBitmapIndexSelector(final Index index)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
}
|
}
|
||||||
|
@ -473,7 +474,7 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
{
|
{
|
||||||
return new Indexed<String>()
|
return new Indexed<String>()
|
||||||
{
|
{
|
||||||
private final String[] dimVals = index.reverseDimLookup.get(dimension);
|
private final String[] dimVals = index.reverseDimLookup.get(dimension.toLowerCase());
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Class<? extends String> getClazz()
|
public Class<? extends String> getClazz()
|
||||||
|
@ -513,22 +514,10 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
||||||
return index.timeOffsets.length;
|
return index.timeOffsets.length;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public int[] getInvertedIndex(String dimension, String value)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Offset getInvertedIndexOffset(String dimension, String value)
|
|
||||||
{
|
|
||||||
return new ArrayBasedOffset(getInvertedIndex(dimension, value));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value)
|
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value)
|
||||||
{
|
{
|
||||||
return index.getInvertedIndex(dimension, value);
|
return index.getInvertedIndex(dimension.toLowerCase(), value);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,8 +30,8 @@ import com.metamx.common.guava.FunctionalIterator;
|
||||||
import com.metamx.druid.BaseStorageAdapter;
|
import com.metamx.druid.BaseStorageAdapter;
|
||||||
import com.metamx.druid.Capabilities;
|
import com.metamx.druid.Capabilities;
|
||||||
import com.metamx.druid.QueryGranularity;
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.index.brita.BitmapIndexSelector;
|
||||||
import com.metamx.druid.index.brita.Filter;
|
import com.metamx.druid.index.brita.Filter;
|
||||||
import com.metamx.druid.index.brita.InvertedIndexSelector;
|
|
||||||
import com.metamx.druid.index.v1.processing.Cursor;
|
import com.metamx.druid.index.v1.processing.Cursor;
|
||||||
import com.metamx.druid.index.v1.processing.DimensionSelector;
|
import com.metamx.druid.index.v1.processing.DimensionSelector;
|
||||||
import com.metamx.druid.index.v1.processing.Offset;
|
import com.metamx.druid.index.v1.processing.Offset;
|
||||||
|
@ -77,7 +77,7 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public int getDimensionCardinality(String dimension)
|
public int getDimensionCardinality(String dimension)
|
||||||
{
|
{
|
||||||
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimension);
|
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimension.toLowerCase());
|
||||||
if (dimValueLookup == null) {
|
if (dimValueLookup == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -127,7 +127,7 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
if (filter == null) {
|
if (filter == null) {
|
||||||
iterable = new NoFilterCursorIterable(index, actualInterval, gran);
|
iterable = new NoFilterCursorIterable(index, actualInterval, gran);
|
||||||
} else {
|
} else {
|
||||||
Offset offset = new ConciseOffset(filter.goConcise(new MMappedInvertedIndexSelector(index)));
|
Offset offset = new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index)));
|
||||||
|
|
||||||
iterable = new CursorIterable(index, actualInterval, gran, offset);
|
iterable = new CursorIterable(index, actualInterval, gran, offset);
|
||||||
}
|
}
|
||||||
|
@ -144,13 +144,13 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimValueLookup(String dimension)
|
public Indexed<String> getDimValueLookup(String dimension)
|
||||||
{
|
{
|
||||||
return index.getDimValueLookup(dimension);
|
return index.getDimValueLookup(dimension.toLowerCase());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal)
|
public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal)
|
||||||
{
|
{
|
||||||
return index.getInvertedIndex(dimension, dimVal);
|
return index.getInvertedIndex(dimension.toLowerCase(), dimVal);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -158,7 +158,7 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
{
|
{
|
||||||
return new ConciseOffset(
|
return new ConciseOffset(
|
||||||
filter.goConcise(
|
filter.goConcise(
|
||||||
new MMappedInvertedIndexSelector(index)
|
new MMappedBitmapIndexSelector(index)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -241,8 +241,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(final String dimensionName)
|
public DimensionSelector makeDimensionSelector(String dimension)
|
||||||
{
|
{
|
||||||
|
final String dimensionName = dimension;
|
||||||
final Indexed<? extends IndexedInts> rowVals = index.getDimColumn(dimensionName);
|
final Indexed<? extends IndexedInts> rowVals = index.getDimColumn(dimensionName);
|
||||||
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimensionName);
|
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimensionName);
|
||||||
|
|
||||||
|
@ -280,8 +281,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatMetricSelector makeFloatMetricSelector(String metricName)
|
public FloatMetricSelector makeFloatMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
IndexedFloats cachedMetricVals = (IndexedFloats) metricHolderCache.get(metricName);
|
IndexedFloats cachedMetricVals = (IndexedFloats) metricHolderCache.get(metricName);
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
if (cachedMetricVals == null) {
|
||||||
|
@ -315,8 +317,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ComplexMetricSelector makeComplexMetricSelector(String metricName)
|
public ComplexMetricSelector makeComplexMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
Indexed cachedMetricVals = (Indexed) metricHolderCache.get(metricName);
|
Indexed cachedMetricVals = (Indexed) metricHolderCache.get(metricName);
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
if (cachedMetricVals == null) {
|
||||||
|
@ -490,8 +493,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DimensionSelector makeDimensionSelector(final String dimensionName)
|
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||||
{
|
{
|
||||||
|
final String dimensionName = dimension.toLowerCase();
|
||||||
final Indexed<? extends IndexedInts> rowVals = index.getDimColumn(dimensionName);
|
final Indexed<? extends IndexedInts> rowVals = index.getDimColumn(dimensionName);
|
||||||
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimensionName);
|
final Indexed<String> dimValueLookup = index.getDimValueLookup(dimensionName);
|
||||||
|
|
||||||
|
@ -529,8 +533,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FloatMetricSelector makeFloatMetricSelector(String metricName)
|
public FloatMetricSelector makeFloatMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
IndexedFloats cachedMetricVals = (IndexedFloats) metricCacheMap.get(metricName);
|
IndexedFloats cachedMetricVals = (IndexedFloats) metricCacheMap.get(metricName);
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
if (cachedMetricVals == null) {
|
||||||
|
@ -566,8 +571,9 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ComplexMetricSelector makeComplexMetricSelector(String metricName)
|
public ComplexMetricSelector makeComplexMetricSelector(String metric)
|
||||||
{
|
{
|
||||||
|
final String metricName = metric.toLowerCase();
|
||||||
Indexed cachedMetricVals = (Indexed) metricCacheMap.get(metricName);
|
Indexed cachedMetricVals = (Indexed) metricCacheMap.get(metricName);
|
||||||
|
|
||||||
if (cachedMetricVals == null) {
|
if (cachedMetricVals == null) {
|
||||||
|
@ -625,11 +631,11 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class MMappedInvertedIndexSelector implements InvertedIndexSelector
|
private static class MMappedBitmapIndexSelector implements BitmapIndexSelector
|
||||||
{
|
{
|
||||||
private final MMappedIndex index;
|
private final MMappedIndex index;
|
||||||
|
|
||||||
public MMappedInvertedIndexSelector(final MMappedIndex index)
|
public MMappedBitmapIndexSelector(final MMappedIndex index)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
}
|
}
|
||||||
|
@ -637,7 +643,7 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getDimensionValues(String dimension)
|
public Indexed<String> getDimensionValues(String dimension)
|
||||||
{
|
{
|
||||||
return index.getDimValueLookup(dimension);
|
return index.getDimValueLookup(dimension.toLowerCase());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -646,22 +652,10 @@ public class MMappedIndexStorageAdapter extends BaseStorageAdapter
|
||||||
return index.getReadOnlyTimestamps().size();
|
return index.getReadOnlyTimestamps().size();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public int[] getInvertedIndex(String dimension, String value)
|
|
||||||
{
|
|
||||||
throw new UnsupportedOperationException();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Offset getInvertedIndexOffset(String dimension, String value)
|
|
||||||
{
|
|
||||||
return new ConciseOffset(index.getInvertedIndex(dimension, value));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value)
|
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value)
|
||||||
{
|
{
|
||||||
return index.getInvertedIndex(dimension, value);
|
return index.getInvertedIndex(dimension.toLowerCase(), value);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,12 +45,7 @@ public class EmptyIndexTest
|
||||||
tmpDir.deleteOnExit();
|
tmpDir.deleteOnExit();
|
||||||
|
|
||||||
IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||||
IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(
|
IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex);
|
||||||
new Interval("2012-08-01/P3D"),
|
|
||||||
emptyIndex,
|
|
||||||
new ArrayList<String>(),
|
|
||||||
new ArrayList<String>()
|
|
||||||
);
|
|
||||||
IndexMerger.merge(Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir);
|
IndexMerger.merge(Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir);
|
||||||
|
|
||||||
MMappedIndex emptyIndexMMapped = IndexIO.mapDir(tmpDir);
|
MMappedIndex emptyIndexMMapped = IndexIO.mapDir(tmpDir);
|
||||||
|
|
|
@ -0,0 +1,113 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||||
|
*
|
||||||
|
* This program is free software; you can redistribute it and/or
|
||||||
|
* modify it under the terms of the GNU General Public License
|
||||||
|
* as published by the Free Software Foundation; either version 2
|
||||||
|
* of the License, or (at your option) any later version.
|
||||||
|
*
|
||||||
|
* This program is distributed in the hope that it will be useful,
|
||||||
|
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||||
|
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||||
|
* GNU General Public License for more details.
|
||||||
|
*
|
||||||
|
* You should have received a copy of the GNU General Public License
|
||||||
|
* along with this program; if not, write to the Free Software
|
||||||
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package com.metamx.druid.index.v1;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.io.Files;
|
||||||
|
import com.metamx.druid.QueryGranularity;
|
||||||
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
|
import junit.framework.Assert;
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class IndexMergerTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testPersistCaseInsensitive() throws Exception
|
||||||
|
{
|
||||||
|
final long timestamp = System.currentTimeMillis();
|
||||||
|
IncrementalIndex toPersist = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
|
||||||
|
|
||||||
|
final File tempDir = Files.createTempDir();
|
||||||
|
try {
|
||||||
|
MMappedIndex index = IndexIO.mapDir(IndexMerger.persist(toPersist, tempDir));
|
||||||
|
|
||||||
|
Assert.assertEquals(2, index.getTimestamps().size());
|
||||||
|
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions()));
|
||||||
|
Assert.assertEquals(0, index.getAvailableMetrics().size());
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
tempDir.delete();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPersistMergeCaseInsensitive() throws Exception
|
||||||
|
{
|
||||||
|
final long timestamp = System.currentTimeMillis();
|
||||||
|
IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
|
||||||
|
|
||||||
|
IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||||
|
|
||||||
|
toPersist2.add(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
timestamp,
|
||||||
|
Arrays.asList("DIm1", "DIM2"),
|
||||||
|
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2", "DIm1", "10000", "DIM2", "100000000")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
toPersist2.add(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
timestamp,
|
||||||
|
Arrays.asList("dIM1", "dIm2"),
|
||||||
|
ImmutableMap.<String, Object>of("DIm1", "1", "DIM2", "2", "dim1", "5", "dim2", "6")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
|
final File tempDir1 = Files.createTempDir();
|
||||||
|
final File tempDir2 = Files.createTempDir();
|
||||||
|
final File mergedDir = Files.createTempDir();
|
||||||
|
try {
|
||||||
|
MMappedIndex index1 = IndexIO.mapDir(IndexMerger.persist(toPersist1, tempDir1));
|
||||||
|
|
||||||
|
Assert.assertEquals(2, index1.getTimestamps().size());
|
||||||
|
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions()));
|
||||||
|
Assert.assertEquals(0, index1.getAvailableMetrics().size());
|
||||||
|
|
||||||
|
MMappedIndex index2 = IndexIO.mapDir(IndexMerger.persist(toPersist2, tempDir2));
|
||||||
|
|
||||||
|
Assert.assertEquals(2, index2.getTimestamps().size());
|
||||||
|
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions()));
|
||||||
|
Assert.assertEquals(0, index2.getAvailableMetrics().size());
|
||||||
|
|
||||||
|
MMappedIndex merged = IndexIO.mapDir(
|
||||||
|
IndexMerger.mergeMMapped(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir)
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(3, merged.getTimestamps().size());
|
||||||
|
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
|
||||||
|
Assert.assertEquals(0, merged.getAvailableMetrics().size());
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
FileUtils.deleteQuietly(tempDir1);
|
||||||
|
FileUtils.deleteQuietly(tempDir2);
|
||||||
|
FileUtils.deleteQuietly(mergedDir);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,16 +19,21 @@
|
||||||
|
|
||||||
package com.metamx.druid.index.v1;
|
package com.metamx.druid.index.v1;
|
||||||
|
|
||||||
|
import com.google.common.base.Charsets;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.io.CharStreams;
|
import com.google.common.io.CharStreams;
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
|
import com.google.common.io.InputSupplier;
|
||||||
import com.google.common.io.LineProcessor;
|
import com.google.common.io.LineProcessor;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
import com.metamx.common.parsers.DelimitedParser;
|
||||||
|
import com.metamx.common.parsers.Parser;
|
||||||
|
import com.metamx.common.parsers.ToLowerCaseParser;
|
||||||
import com.metamx.druid.QueryGranularity;
|
import com.metamx.druid.QueryGranularity;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
import com.metamx.druid.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
@ -36,7 +41,11 @@ import com.metamx.druid.client.RangeIterable;
|
||||||
import com.metamx.druid.guava.GuavaUtils;
|
import com.metamx.druid.guava.GuavaUtils;
|
||||||
import com.metamx.druid.index.v1.serde.ComplexMetricSerde;
|
import com.metamx.druid.index.v1.serde.ComplexMetricSerde;
|
||||||
import com.metamx.druid.index.v1.serde.ComplexMetrics;
|
import com.metamx.druid.index.v1.serde.ComplexMetrics;
|
||||||
|
import com.metamx.druid.indexer.data.DelimitedDataSpec;
|
||||||
|
import com.metamx.druid.indexer.data.StringInputRowParser;
|
||||||
|
import com.metamx.druid.indexer.data.TimestampSpec;
|
||||||
import com.metamx.druid.input.InputRow;
|
import com.metamx.druid.input.InputRow;
|
||||||
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
import com.metamx.druid.kv.ArrayIndexed;
|
import com.metamx.druid.kv.ArrayIndexed;
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import com.metamx.druid.kv.IndexedFloats;
|
import com.metamx.druid.kv.IndexedFloats;
|
||||||
|
@ -50,6 +59,7 @@ import org.joda.time.Interval;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.nio.ByteOrder;
|
import java.nio.ByteOrder;
|
||||||
import java.nio.FloatBuffer;
|
import java.nio.FloatBuffer;
|
||||||
|
@ -71,8 +81,9 @@ public class TestIndex
|
||||||
private static MMappedIndex mmappedIndex = null;
|
private static MMappedIndex mmappedIndex = null;
|
||||||
private static MMappedIndex mergedRealtime = null;
|
private static MMappedIndex mergedRealtime = null;
|
||||||
|
|
||||||
public static final String[] DIMENSIONS = new String[]{"provider", "quality", "placement", "placementish"};
|
public static final String[] COLUMNS = new String[]{"ts", "provider", "quALIty", "plAcEmEnT", "pLacementish", "iNdEx"};
|
||||||
public static final String[] METRICS = new String[]{"index"};
|
public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"};
|
||||||
|
public static final String[] METRICS = new String[]{"iNdEx"};
|
||||||
public static final Map<String, Integer> dimIds = Maps.uniqueIndex(
|
public static final Map<String, Integer> dimIds = Maps.uniqueIndex(
|
||||||
new RangeIterable(4),
|
new RangeIterable(4),
|
||||||
new Function<Integer, String>()
|
new Function<Integer, String>()
|
||||||
|
@ -310,7 +321,9 @@ public class TestIndex
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
com.metamx.druid.index.v1.IndexIO.mapDir(topFile),
|
com.metamx.druid.index.v1.IndexIO.mapDir(topFile),
|
||||||
com.metamx.druid.index.v1.IndexIO.mapDir(bottomFile)
|
com.metamx.druid.index.v1.IndexIO.mapDir(bottomFile)
|
||||||
), METRIC_AGGS, mergedFile
|
),
|
||||||
|
METRIC_AGGS,
|
||||||
|
mergedFile
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -324,9 +337,8 @@ public class TestIndex
|
||||||
|
|
||||||
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
|
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
|
||||||
{
|
{
|
||||||
URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
|
final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
|
||||||
String filename = resource.getFile();
|
log.info("Realtime loading index file[%s]", resource);
|
||||||
log.info("Realtime loading index file[%s]", filename);
|
|
||||||
|
|
||||||
final IncrementalIndex retVal = new IncrementalIndex(
|
final IncrementalIndex retVal = new IncrementalIndex(
|
||||||
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS
|
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS
|
||||||
|
@ -336,9 +348,24 @@ public class TestIndex
|
||||||
int lineCount;
|
int lineCount;
|
||||||
try {
|
try {
|
||||||
lineCount = CharStreams.readLines(
|
lineCount = CharStreams.readLines(
|
||||||
GuavaUtils.joinFiles(new File(filename)),
|
CharStreams.newReaderSupplier(
|
||||||
|
new InputSupplier<InputStream>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public InputStream getInput() throws IOException
|
||||||
|
{
|
||||||
|
return resource.openStream();
|
||||||
|
}
|
||||||
|
},
|
||||||
|
Charsets.UTF_8
|
||||||
|
),
|
||||||
new LineProcessor<Integer>()
|
new LineProcessor<Integer>()
|
||||||
{
|
{
|
||||||
|
StringInputRowParser parser = new StringInputRowParser(
|
||||||
|
new TimestampSpec("ts", "iso"),
|
||||||
|
new DelimitedDataSpec("\t", Arrays.asList(COLUMNS), Arrays.asList(DIMENSIONS)),
|
||||||
|
Arrays.<String>asList()
|
||||||
|
);
|
||||||
boolean runOnce = false;
|
boolean runOnce = false;
|
||||||
int lineCount = 0;
|
int lineCount = 0;
|
||||||
|
|
||||||
|
@ -352,35 +379,7 @@ public class TestIndex
|
||||||
|
|
||||||
final String[] splits = line.split("\t");
|
final String[] splits = line.split("\t");
|
||||||
|
|
||||||
retVal.add(
|
retVal.add(parser.parse(line));
|
||||||
new InputRow()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public long getTimestampFromEpoch()
|
|
||||||
{
|
|
||||||
return new DateTime(splits[0]).getMillis();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<String> getDimensions()
|
|
||||||
{
|
|
||||||
return Arrays.asList(DIMENSIONS);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<String> getDimension(String dimension)
|
|
||||||
{
|
|
||||||
return Arrays.asList(splits[dimIds.get(dimension) + 1].split("\u0001"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public float getFloatMetric(String metric)
|
|
||||||
{
|
|
||||||
Preconditions.checkArgument(METRICS[0].equals(metric), "WTF!?");
|
|
||||||
return Float.parseFloat(splits[5]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
++lineCount;
|
++lineCount;
|
||||||
return true;
|
return true;
|
||||||
|
|
|
@ -54,7 +54,7 @@ public class QueryRunnerTestHelper
|
||||||
public static final String dataSource = "testing";
|
public static final String dataSource = "testing";
|
||||||
public static final QueryGranularity gran = QueryGranularity.DAY;
|
public static final QueryGranularity gran = QueryGranularity.DAY;
|
||||||
public static final QueryGranularity allGran = QueryGranularity.ALL;
|
public static final QueryGranularity allGran = QueryGranularity.ALL;
|
||||||
public static final String providerDimension = "provider";
|
public static final String providerDimension = "proVider";
|
||||||
public static final String qualityDimension = "quality";
|
public static final String qualityDimension = "quality";
|
||||||
public static final String placementishDimension = "placementish";
|
public static final String placementishDimension = "placementish";
|
||||||
public static final String indexMetric = "index";
|
public static final String indexMetric = "index";
|
||||||
|
|
|
@ -21,6 +21,7 @@ package com.metamx.druid.query.search;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.guava.Sequences;
|
import com.metamx.common.guava.Sequences;
|
||||||
import com.metamx.druid.Druids;
|
import com.metamx.druid.Druids;
|
||||||
|
@ -42,6 +43,7 @@ import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -75,12 +77,12 @@ public class SearchQueryRunnerTest
|
||||||
.query("a")
|
.query("a")
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(
|
expectedResults.put(
|
||||||
QueryRunnerTestHelper.qualityDimension,
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment")
|
Sets.newHashSet("automotive", "mezzanine", "travel", "health", "entertainment")
|
||||||
);
|
);
|
||||||
expectedResults.put(QueryRunnerTestHelper.providerDimension, Sets.newHashSet("total_market"));
|
expectedResults.put(QueryRunnerTestHelper.providerDimension.toLowerCase(), Sets.newHashSet("total_market"));
|
||||||
expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a"));
|
expectedResults.put(QueryRunnerTestHelper.placementishDimension, Sets.newHashSet("a"));
|
||||||
|
|
||||||
checkSearchQuery(searchQuery, expectedResults);
|
checkSearchQuery(searchQuery, expectedResults);
|
||||||
|
@ -96,7 +98,7 @@ public class SearchQueryRunnerTest
|
||||||
.query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve"), null))
|
.query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve"), null))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive"));
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, Sets.newHashSet("automotive"));
|
||||||
|
|
||||||
checkSearchQuery(searchQuery, expectedResults);
|
checkSearchQuery(searchQuery, expectedResults);
|
||||||
|
@ -129,7 +131,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithDimension2()
|
public void testSearchWithDimension2()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
checkSearchQuery(
|
checkSearchQuery(
|
||||||
|
@ -147,7 +149,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithDimensions1()
|
public void testSearchWithDimensions1()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.putAll(
|
expectedResults.putAll(
|
||||||
ImmutableMap.<String, Set<String>>of(
|
ImmutableMap.<String, Set<String>>of(
|
||||||
QueryRunnerTestHelper.qualityDimension,
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
@ -167,7 +169,12 @@ public class SearchQueryRunnerTest
|
||||||
Druids.newSearchQueryBuilder()
|
Druids.newSearchQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.granularity(QueryRunnerTestHelper.allGran)
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
.dimensions(Arrays.asList(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.providerDimension))
|
.dimensions(
|
||||||
|
Arrays.asList(
|
||||||
|
QueryRunnerTestHelper.qualityDimension,
|
||||||
|
QueryRunnerTestHelper.providerDimension
|
||||||
|
)
|
||||||
|
)
|
||||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
.query("a")
|
.query("a")
|
||||||
.build(),
|
.build(),
|
||||||
|
@ -178,14 +185,19 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithDimensions2()
|
public void testSearchWithDimensions2()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
checkSearchQuery(
|
checkSearchQuery(
|
||||||
Druids.newSearchQueryBuilder()
|
Druids.newSearchQueryBuilder()
|
||||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.granularity(QueryRunnerTestHelper.allGran)
|
.granularity(QueryRunnerTestHelper.allGran)
|
||||||
.dimensions(Arrays.asList(QueryRunnerTestHelper.placementishDimension, QueryRunnerTestHelper.providerDimension))
|
.dimensions(
|
||||||
|
Arrays.asList(
|
||||||
|
QueryRunnerTestHelper.placementishDimension,
|
||||||
|
QueryRunnerTestHelper.providerDimension
|
||||||
|
)
|
||||||
|
)
|
||||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||||
.query("mark")
|
.query("mark")
|
||||||
.build(),
|
.build(),
|
||||||
|
@ -196,7 +208,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithSingleFilter1()
|
public void testSearchWithSingleFilter1()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(
|
expectedResults.put(
|
||||||
QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive"))
|
QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive"))
|
||||||
);
|
);
|
||||||
|
@ -217,7 +229,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithSingleFilter2()
|
public void testSearchWithSingleFilter2()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet<String>(Arrays.asList("total_market")));
|
||||||
|
|
||||||
checkSearchQuery(
|
checkSearchQuery(
|
||||||
|
@ -236,7 +248,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchMultiAndFilter()
|
public void testSearchMultiAndFilter()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
||||||
|
|
||||||
DimFilter filter = Druids.newAndDimFilterBuilder()
|
DimFilter filter = Druids.newAndDimFilterBuilder()
|
||||||
|
@ -270,7 +282,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithMultiOrFilter()
|
public void testSearchWithMultiOrFilter()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
expectedResults.put(QueryRunnerTestHelper.qualityDimension, new HashSet<String>(Arrays.asList("automotive")));
|
||||||
|
|
||||||
DimFilter filter = Druids.newOrDimFilterBuilder()
|
DimFilter filter = Druids.newOrDimFilterBuilder()
|
||||||
|
@ -304,7 +316,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithEmptyResults()
|
public void testSearchWithEmptyResults()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
|
|
||||||
checkSearchQuery(
|
checkSearchQuery(
|
||||||
Druids.newSearchQueryBuilder()
|
Druids.newSearchQueryBuilder()
|
||||||
|
@ -320,7 +332,7 @@ public class SearchQueryRunnerTest
|
||||||
@Test
|
@Test
|
||||||
public void testSearchWithFilterEmptyResults()
|
public void testSearchWithFilterEmptyResults()
|
||||||
{
|
{
|
||||||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||||
|
|
||||||
DimFilter filter = Druids.newAndDimFilterBuilder()
|
DimFilter filter = Druids.newAndDimFilterBuilder()
|
||||||
.fields(
|
.fields(
|
||||||
|
@ -364,7 +376,10 @@ public class SearchQueryRunnerTest
|
||||||
for (SearchHit resultValue : resultValues) {
|
for (SearchHit resultValue : resultValues) {
|
||||||
String dimension = resultValue.getDimension();
|
String dimension = resultValue.getDimension();
|
||||||
String theValue = resultValue.getValue();
|
String theValue = resultValue.getValue();
|
||||||
Assert.assertTrue(expectedResults.containsKey(dimension));
|
Assert.assertTrue(
|
||||||
|
String.format("Result had unknown dimension[%s]", dimension),
|
||||||
|
expectedResults.containsKey(dimension)
|
||||||
|
);
|
||||||
|
|
||||||
Set<String> expectedSet = expectedResults.get(dimension);
|
Set<String> expectedSet = expectedResults.get(dimension);
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
|
|
|
@ -61,9 +61,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Collection<?> constructorFeeder() throws IOException
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
return QueryRunnerTestHelper.makeQueryRunners(
|
return QueryRunnerTestHelper.makeQueryRunners(new TimeseriesQueryRunnerFactory());
|
||||||
new TimeseriesQueryRunnerFactory()
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private final QueryRunner runner;
|
private final QueryRunner runner;
|
||||||
|
|
Loading…
Reference in New Issue