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:
Eric Tschetter 2012-11-19 17:01:17 -06:00
parent aa073e4e73
commit 701cc9562b
49 changed files with 578 additions and 309 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -45,7 +45,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
) )
{ {
this.name = name; this.name = name;
this.fieldName = fieldName.toLowerCase(); this.fieldName = fieldName;
} }
@Override @Override

View File

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

View File

@ -45,7 +45,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory
) )
{ {
this.name = name; this.name = name;
this.fieldName = fieldName.toLowerCase(); this.fieldName = fieldName;
} }
@Override @Override

View File

@ -45,7 +45,7 @@ public class MaxAggregatorFactory implements AggregatorFactory
) )
{ {
this.name = name; this.name = name;
this.fieldName = fieldName.toLowerCase(); this.fieldName = fieldName;
} }
@Override @Override

View File

@ -45,7 +45,7 @@ public class MinAggregatorFactory implements AggregatorFactory
) )
{ {
this.name = name; this.name = name;
this.fieldName = fieldName.toLowerCase(); this.fieldName = fieldName;
} }
@Override @Override

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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