mirror of https://github.com/apache/druid.git
Merge pull request #891 from metamx/fix-casing
Make Druid case sensitive
This commit is contained in:
commit
f2d94eecde
|
@ -70,7 +70,7 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory
|
|||
)
|
||||
{
|
||||
this.name = name;
|
||||
this.fieldName = fieldName.toLowerCase();
|
||||
this.fieldName = fieldName;
|
||||
this.resolution = resolution == null ? ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE : resolution;
|
||||
this.numBuckets = numBuckets == null ? ApproximateHistogram.DEFAULT_BUCKET_SIZE : numBuckets;
|
||||
this.lowerLimit = lowerLimit == null ? Float.NEGATIVE_INFINITY : lowerLimit;
|
||||
|
|
|
@ -160,7 +160,7 @@ public class ApproximateHistogramGroupByQueryTest
|
|||
Arrays.<DimensionSpec>asList(
|
||||
new DefaultDimensionSpec(
|
||||
QueryRunnerTestHelper.marketDimension,
|
||||
"marKetAlias"
|
||||
"marketalias"
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -169,7 +169,7 @@ public class ApproximateHistogramGroupByQueryTest
|
|||
new DefaultLimitSpec(
|
||||
Lists.newArrayList(
|
||||
new OrderByColumnSpec(
|
||||
"marKetAlias",
|
||||
"marketalias",
|
||||
OrderByColumnSpec.Direction.DESCENDING
|
||||
)
|
||||
), 1
|
||||
|
|
|
@ -307,7 +307,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
|
||||
for (AggregatorFactory factory : config.getSchema().getDataSchema().getAggregators()) {
|
||||
metricNames.add(factory.getName().toLowerCase());
|
||||
metricNames.add(factory.getName());
|
||||
}
|
||||
|
||||
parser = config.getParser();
|
||||
|
|
|
@ -64,7 +64,7 @@ public abstract class AbstractTask implements Task
|
|||
this.id = Preconditions.checkNotNull(id, "id");
|
||||
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
||||
this.taskResource = Preconditions.checkNotNull(taskResource, "resource");
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource.toLowerCase(), "dataSource");
|
||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -270,7 +270,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
|
|||
return String.format(
|
||||
"%s_%s",
|
||||
dataSource,
|
||||
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase()
|
||||
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -78,7 +78,7 @@ public class MergeTaskBaseTest
|
|||
+ "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0"
|
||||
+ "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0"
|
||||
, Charsets.UTF_8
|
||||
).toString().toLowerCase() + "_";
|
||||
).toString() + "_";
|
||||
Assert.assertEquals(
|
||||
desiredPrefix,
|
||||
testMergeTaskBase.getId().substring(0, desiredPrefix.length())
|
||||
|
|
2
pom.xml
2
pom.xml
|
@ -42,7 +42,7 @@
|
|||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.26.9</metamx.java-util.version>
|
||||
<apache.curator.version>2.6.0</apache.curator.version>
|
||||
<druid.api.version>0.2.18-SNAPSHOT</druid.api.version>
|
||||
<druid.api.version>0.3.0</druid.api.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
|
|
|
@ -36,7 +36,7 @@ public class TableDataSource implements DataSource
|
|||
@JsonCreator
|
||||
public TableDataSource(@JsonProperty("name") String name)
|
||||
{
|
||||
this.name = (name == null ? null : name.toLowerCase());
|
||||
this.name = (name == null ? null : name);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -1,141 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 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 io.druid.query.aggregation;
|
||||
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ToLowerCaseAggregatorFactory implements AggregatorFactory
|
||||
{
|
||||
private final AggregatorFactory baseAggregatorFactory;
|
||||
|
||||
public ToLowerCaseAggregatorFactory(AggregatorFactory baseAggregatorFactory)
|
||||
{
|
||||
this.baseAggregatorFactory = baseAggregatorFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return baseAggregatorFactory.factorize(metricFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return baseAggregatorFactory.factorizeBuffered(metricFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return baseAggregatorFactory.getComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
{
|
||||
return baseAggregatorFactory.combine(lhs, rhs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
return baseAggregatorFactory.getCombiningFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return baseAggregatorFactory.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
return baseAggregatorFactory.deserialize(object);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
{
|
||||
return baseAggregatorFactory.finalizeComputation(object);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return baseAggregatorFactory.getName().toLowerCase();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> requiredFields()
|
||||
{
|
||||
return baseAggregatorFactory.requiredFields();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return baseAggregatorFactory.getCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTypeName()
|
||||
{
|
||||
return baseAggregatorFactory.getTypeName();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxIntermediateSize()
|
||||
{
|
||||
return baseAggregatorFactory.getMaxIntermediateSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getAggregatorStartValue()
|
||||
{
|
||||
return baseAggregatorFactory.getAggregatorStartValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
ToLowerCaseAggregatorFactory that = (ToLowerCaseAggregatorFactory) o;
|
||||
|
||||
if (baseAggregatorFactory != null ? !baseAggregatorFactory.equals(that.baseAggregatorFactory) : that.baseAggregatorFactory != null)
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return baseAggregatorFactory != null ? baseAggregatorFactory.hashCode() : 0;
|
||||
}
|
||||
}
|
|
@ -61,7 +61,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
|||
)
|
||||
{
|
||||
this.name = name;
|
||||
this.fieldName = fieldName.toLowerCase();
|
||||
this.fieldName = fieldName;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -173,7 +173,7 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
public String apply(Row input)
|
||||
{
|
||||
// Multi-value dimensions have all been flattened at this point;
|
||||
final List<String> dimList = input.getDimension(dimension.toLowerCase());
|
||||
final List<String> dimList = input.getDimension(dimension);
|
||||
return dimList.isEmpty() ? null : dimList.get(0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -108,7 +108,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
}
|
||||
|
||||
for (String dimension : dimsToSearch) {
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -22,8 +22,6 @@ package io.druid.query.search.search;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
@ -41,17 +39,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
@JsonProperty("values") List<String> values
|
||||
)
|
||||
{
|
||||
this.values = Lists.transform(
|
||||
values,
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String s)
|
||||
{
|
||||
return s.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -64,7 +52,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
public boolean accept(String dimVal)
|
||||
{
|
||||
for (String value : values) {
|
||||
if (dimVal == null || !dimVal.toLowerCase().contains(value)) {
|
||||
if (dimVal == null || !dimVal.toLowerCase().contains(value.toLowerCase())) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -97,19 +85,25 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
|
|||
public String toString()
|
||||
{
|
||||
return "FragmentSearchQuerySpec{" +
|
||||
"values=" + values +
|
||||
"values=" + values +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
FragmentSearchQuerySpec that = (FragmentSearchQuerySpec) o;
|
||||
|
||||
if (values != null ? !values.equals(that.values) : that.values != null) return false;
|
||||
if (values != null ? !values.equals(that.values) : that.values != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
|||
@JsonProperty("value") String value
|
||||
)
|
||||
{
|
||||
this.value = value.toLowerCase();
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -53,7 +53,7 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
|||
if (dimVal == null) {
|
||||
return false;
|
||||
}
|
||||
return dimVal.toLowerCase().contains(value);
|
||||
return dimVal.toLowerCase().contains(value.toLowerCase());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -78,12 +78,18 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
InsensitiveContainsSearchQuerySpec that = (InsensitiveContainsSearchQuerySpec) o;
|
||||
|
||||
if (value != null ? !value.equals(that.value) : that.value != null) return false;
|
||||
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public Indexed<String> getDimensionValues(String dimension)
|
||||
{
|
||||
final Column columnDesc = index.getColumn(dimension.toLowerCase());
|
||||
final Column columnDesc = index.getColumn(dimension);
|
||||
if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) {
|
||||
return null;
|
||||
}
|
||||
|
@ -112,7 +112,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null) {
|
||||
return bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
|
@ -126,7 +126,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, int idx)
|
||||
{
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null || column.getCapabilities() == null) {
|
||||
bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
|
@ -143,7 +143,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column == null || !column.getCapabilities().hasSpatialIndexes()) {
|
||||
return new ImmutableRTree();
|
||||
}
|
||||
|
|
|
@ -684,7 +684,7 @@ public class IndexIO
|
|||
);
|
||||
}
|
||||
columns.put(
|
||||
dimension.toLowerCase(),
|
||||
dimension,
|
||||
builder.build()
|
||||
);
|
||||
}
|
||||
|
@ -693,7 +693,7 @@ public class IndexIO
|
|||
final MetricHolder metricHolder = index.getMetricHolder(metric);
|
||||
if (metricHolder.getType() == MetricHolder.MetricType.FLOAT) {
|
||||
columns.put(
|
||||
metric.toLowerCase(),
|
||||
metric,
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.FLOAT)
|
||||
.setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType, BYTE_ORDER))
|
||||
|
@ -701,7 +701,7 @@ public class IndexIO
|
|||
);
|
||||
} else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) {
|
||||
columns.put(
|
||||
metric.toLowerCase(),
|
||||
metric,
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.COMPLEX)
|
||||
.setComplexColumn(
|
||||
|
@ -716,10 +716,10 @@ public class IndexIO
|
|||
|
||||
Set<String> colSet = Sets.newTreeSet();
|
||||
for (String dimension : index.getAvailableDimensions()) {
|
||||
colSet.add(dimension.toLowerCase());
|
||||
colSet.add(dimension);
|
||||
}
|
||||
for (String metric : index.getAvailableMetrics()) {
|
||||
colSet.add(metric.toLowerCase());
|
||||
colSet.add(metric);
|
||||
}
|
||||
|
||||
String[] cols = colSet.toArray(new String[colSet.size()]);
|
||||
|
|
|
@ -57,7 +57,6 @@ import io.druid.common.utils.SerializerUtils;
|
|||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import io.druid.segment.column.ColumnDescriptor;
|
||||
|
@ -185,7 +184,13 @@ public class IndexMaker
|
|||
|
||||
log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
|
||||
return merge(
|
||||
Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index, bitmapSerdeFactory.getBitmapFactory())),
|
||||
Arrays.<IndexableAdapter>asList(
|
||||
new IncrementalIndexAdapter(
|
||||
dataInterval,
|
||||
index,
|
||||
bitmapSerdeFactory.getBitmapFactory()
|
||||
)
|
||||
),
|
||||
index.getMetricAggs(),
|
||||
outDir,
|
||||
progress
|
||||
|
@ -243,11 +248,6 @@ public class IndexMaker
|
|||
throw new ISE("Couldn't make outdir[%s].", outDir);
|
||||
}
|
||||
|
||||
final AggregatorFactory[] lowerCaseMetricAggs = new AggregatorFactory[metricAggs.length];
|
||||
for (int i = 0; i < metricAggs.length; i++) {
|
||||
lowerCaseMetricAggs[i] = new ToLowerCaseAggregatorFactory(metricAggs[i]);
|
||||
}
|
||||
|
||||
final List<String> mergedDimensions = mergeIndexed(
|
||||
Lists.transform(
|
||||
adapters,
|
||||
|
@ -256,17 +256,7 @@ public class IndexMaker
|
|||
@Override
|
||||
public Iterable<String> apply(IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getDimensionNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
return input.getDimensionNames();
|
||||
}
|
||||
}
|
||||
)
|
||||
|
@ -283,21 +273,11 @@ public class IndexMaker
|
|||
@Override
|
||||
public Iterable<String> apply(IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getMetricNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
return input.getMetricNames();
|
||||
}
|
||||
}
|
||||
)
|
||||
.concat(Arrays.<Iterable<String>>asList(new AggFactoryStringIndexed(lowerCaseMetricAggs)))
|
||||
.concat(Arrays.<Iterable<String>>asList(new AggFactoryStringIndexed(metricAggs)))
|
||||
)
|
||||
),
|
||||
new Function<String, String>()
|
||||
|
@ -305,17 +285,17 @@ public class IndexMaker
|
|||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
if (mergedMetrics.size() != lowerCaseMetricAggs.length) {
|
||||
throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), lowerCaseMetricAggs.length);
|
||||
if (mergedMetrics.size() != metricAggs.length) {
|
||||
throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), metricAggs.length);
|
||||
}
|
||||
|
||||
final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()];
|
||||
for (int i = 0; i < lowerCaseMetricAggs.length; i++) {
|
||||
AggregatorFactory metricAgg = lowerCaseMetricAggs[i];
|
||||
for (int i = 0; i < metricAggs.length; i++) {
|
||||
AggregatorFactory metricAgg = metricAggs[i];
|
||||
sortedMetricAggs[mergedMetrics.indexOf(metricAgg.getName())] = metricAgg;
|
||||
}
|
||||
|
||||
|
@ -324,7 +304,7 @@ public class IndexMaker
|
|||
throw new IAE(
|
||||
"Metric mismatch, index[%d] [%s] != [%s]",
|
||||
i,
|
||||
lowerCaseMetricAggs[i].getName(),
|
||||
metricAggs[i].getName(),
|
||||
mergedMetrics.get(i)
|
||||
);
|
||||
}
|
||||
|
@ -385,7 +365,7 @@ public class IndexMaker
|
|||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -408,7 +388,7 @@ public class IndexMaker
|
|||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -648,7 +628,7 @@ public class IndexMaker
|
|||
final int[] dimLookup = new int[mergedDimensions.size()];
|
||||
int count = 0;
|
||||
for (String dim : adapter.getDimensionNames()) {
|
||||
dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase());
|
||||
dimLookup[count] = mergedDimensions.indexOf(dim);
|
||||
count++;
|
||||
}
|
||||
|
||||
|
|
|
@ -59,7 +59,6 @@ import io.druid.common.utils.SerializerUtils;
|
|||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import io.druid.segment.column.ValueType;
|
||||
|
@ -240,11 +239,6 @@ public class IndexMerger
|
|||
throw new ISE("Couldn't make outdir[%s].", outDir);
|
||||
}
|
||||
|
||||
final AggregatorFactory[] lowerCaseMetricAggs = new AggregatorFactory[metricAggs.length];
|
||||
for (int i = 0; i < metricAggs.length; i++) {
|
||||
lowerCaseMetricAggs[i] = new ToLowerCaseAggregatorFactory(metricAggs[i]);
|
||||
}
|
||||
|
||||
final List<String> mergedDimensions = mergeIndexed(
|
||||
Lists.transform(
|
||||
indexes,
|
||||
|
@ -253,24 +247,14 @@ public class IndexMerger
|
|||
@Override
|
||||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getDimensionNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(@Nullable String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
return input.getDimensionNames();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
final List<String> mergedMetrics = Lists.transform(
|
||||
mergeIndexed(
|
||||
Lists.<Iterable<String>>newArrayList(
|
||||
Lists.newArrayList(
|
||||
FunctionalIterable
|
||||
.create(indexes)
|
||||
.transform(
|
||||
|
@ -279,21 +263,11 @@ public class IndexMerger
|
|||
@Override
|
||||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getMetricNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(@Nullable String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
return input.getMetricNames();
|
||||
}
|
||||
}
|
||||
)
|
||||
.concat(Arrays.<Iterable<String>>asList(new AggFactoryStringIndexed(lowerCaseMetricAggs)))
|
||||
.concat(Arrays.<Iterable<String>>asList(new AggFactoryStringIndexed(metricAggs)))
|
||||
)
|
||||
),
|
||||
new Function<String, String>()
|
||||
|
@ -301,17 +275,17 @@ public class IndexMerger
|
|||
@Override
|
||||
public String apply(@Nullable String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
if (mergedMetrics.size() != lowerCaseMetricAggs.length) {
|
||||
throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), lowerCaseMetricAggs.length);
|
||||
if (mergedMetrics.size() != metricAggs.length) {
|
||||
throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), metricAggs.length);
|
||||
}
|
||||
|
||||
final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()];
|
||||
for (int i = 0; i < lowerCaseMetricAggs.length; i++) {
|
||||
AggregatorFactory metricAgg = lowerCaseMetricAggs[i];
|
||||
for (int i = 0; i < metricAggs.length; i++) {
|
||||
AggregatorFactory metricAgg = metricAggs[i];
|
||||
sortedMetricAggs[mergedMetrics.indexOf(metricAgg.getName())] = metricAgg;
|
||||
}
|
||||
|
||||
|
@ -320,7 +294,7 @@ public class IndexMerger
|
|||
throw new IAE(
|
||||
"Metric mismatch, index[%d] [%s] != [%s]",
|
||||
i,
|
||||
lowerCaseMetricAggs[i].getName(),
|
||||
metricAggs[i].getName(),
|
||||
mergedMetrics.get(i)
|
||||
);
|
||||
}
|
||||
|
@ -378,7 +352,7 @@ public class IndexMerger
|
|||
@Override
|
||||
public String apply(@Nullable String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -401,7 +375,7 @@ public class IndexMerger
|
|||
@Override
|
||||
public String apply(@Nullable String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
return input;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -604,7 +578,7 @@ public class IndexMerger
|
|||
final int[] dimLookup = new int[mergedDimensions.size()];
|
||||
int count = 0;
|
||||
for (String dim : adapter.getDimensionNames()) {
|
||||
dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase());
|
||||
dimLookup[count] = mergedDimensions.indexOf(dim);
|
||||
count++;
|
||||
}
|
||||
|
||||
|
|
|
@ -91,7 +91,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return 0;
|
||||
}
|
||||
|
||||
Column column = index.getColumn(dimension.toLowerCase());
|
||||
Column column = index.getColumn(dimension);
|
||||
if (column == null) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -267,14 +267,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public DimensionSelector makeDimensionSelector(String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
|
||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName);
|
||||
final Column columnDesc = index.getColumn(dimensionName);
|
||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimension);
|
||||
final Column columnDesc = index.getColumn(dimension);
|
||||
|
||||
if (cachedColumn == null && columnDesc != null) {
|
||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||
dictionaryColumnCache.put(dimensionName, cachedColumn);
|
||||
dictionaryColumnCache.put(dimension, cachedColumn);
|
||||
}
|
||||
|
||||
final DictionaryEncodedColumn column = cachedColumn;
|
||||
|
@ -362,15 +360,14 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
GenericColumn cachedMetricVals = genericColumnCache.get(metricName);
|
||||
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
||||
|
||||
if (cachedMetricVals == null) {
|
||||
Column holder = index.getColumn(metricName);
|
||||
Column holder = index.getColumn(columnName);
|
||||
if (holder != null && (holder.getCapabilities().getType() == ValueType.FLOAT
|
||||
|| holder.getCapabilities().getType() == ValueType.LONG)) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
genericColumnCache.put(columnName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -399,15 +396,14 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public LongColumnSelector makeLongColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
GenericColumn cachedMetricVals = genericColumnCache.get(metricName);
|
||||
GenericColumn cachedMetricVals = genericColumnCache.get(columnName);
|
||||
|
||||
if (cachedMetricVals == null) {
|
||||
Column holder = index.getColumn(metricName);
|
||||
Column holder = index.getColumn(columnName);
|
||||
if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG
|
||||
|| holder.getCapabilities().getType() == ValueType.FLOAT)) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
genericColumnCache.put(columnName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -436,12 +432,10 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||
{
|
||||
final String columnName = column.toLowerCase();
|
||||
|
||||
Object cachedColumnVals = objectColumnCache.get(columnName);
|
||||
Object cachedColumnVals = objectColumnCache.get(column);
|
||||
|
||||
if (cachedColumnVals == null) {
|
||||
Column holder = index.getColumn(columnName);
|
||||
Column holder = index.getColumn(column);
|
||||
|
||||
if (holder != null) {
|
||||
final ColumnCapabilities capabilities = holder.getCapabilities();
|
||||
|
@ -456,7 +450,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
if (cachedColumnVals != null) {
|
||||
objectColumnCache.put(columnName, cachedColumnVals);
|
||||
objectColumnCache.put(column, cachedColumnVals);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -130,7 +130,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
new ColumnSelectorFactory()
|
||||
{
|
||||
@Override
|
||||
public LongColumnSelector makeLongColumnSelector(String columnName)
|
||||
public LongColumnSelector makeLongColumnSelector(final String columnName)
|
||||
{
|
||||
if(columnName.equals(Column.TIME_COLUMN_NAME)){
|
||||
return new LongColumnSelector()
|
||||
|
@ -142,36 +142,33 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
}
|
||||
};
|
||||
}
|
||||
final String metricName = columnName.toLowerCase();
|
||||
return new LongColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public long get()
|
||||
{
|
||||
return in.get().getLongMetric(metricName);
|
||||
return in.get().getLongMetric(columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||
public FloatColumnSelector makeFloatColumnSelector(final String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
return new FloatColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public float get()
|
||||
{
|
||||
return in.get().getFloatMetric(metricName);
|
||||
return in.get().getFloatMetric(columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||
public ObjectColumnSelector makeObjectColumnSelector(final String column)
|
||||
{
|
||||
final String typeName = agg.getTypeName();
|
||||
final String columnName = column.toLowerCase();
|
||||
|
||||
final ObjectColumnSelector<Object> rawColumnSelector = new ObjectColumnSelector<Object>()
|
||||
{
|
||||
|
@ -184,7 +181,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return in.get().getRaw(columnName);
|
||||
return in.get().getRaw(column);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -212,7 +209,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return extractor.extractValue(in.get(), columnName);
|
||||
return extractor.extractValue(in.get(), column);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -221,13 +218,12 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
@Override
|
||||
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
final List<String> dimensionValues = in.get().getDimension(dimensionName);
|
||||
final List<String> dimensionValues = in.get().getDimension(dimension);
|
||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
||||
if (dimensionValues != null) {
|
||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
||||
|
@ -266,13 +262,13 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return in.get().getDimension(dimensionName).get(id);
|
||||
return in.get().getDimension(dimension).get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return in.get().getDimension(dimensionName).indexOf(name);
|
||||
return in.get().getDimension(dimension).indexOf(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -280,7 +276,7 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
);
|
||||
aggPositionOffsets[i] = currAggSize;
|
||||
currAggSize += agg.getMaxIntermediateSize();
|
||||
final String metricName = metrics[i].getName().toLowerCase();
|
||||
final String metricName = metrics[i].getName();
|
||||
metricNamesBuilder.add(metricName);
|
||||
metricIndexesBuilder.put(metricName, i);
|
||||
metricTypesBuilder.put(metricName, metrics[i].getTypeName());
|
||||
|
@ -414,7 +410,6 @@ public class IncrementalIndex implements Iterable<Row>, Closeable
|
|||
synchronized (dimensionOrder) {
|
||||
dims = new String[dimensionOrder.size()][];
|
||||
for (String dimension : rowDimensions) {
|
||||
dimension = dimension.toLowerCase();
|
||||
List<String> dimensionValues = row.getDimension(dimension);
|
||||
|
||||
// Set column capabilities as data is coming in
|
||||
|
|
|
@ -99,7 +99,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public int getDimensionCardinality(String dimension)
|
||||
{
|
||||
IncrementalIndex.DimDim dimDim = index.getDimension(dimension.toLowerCase());
|
||||
IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
||||
if (dimDim == null) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -268,14 +268,13 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public DimensionSelector makeDimensionSelector(String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
final IncrementalIndex.DimDim dimValLookup = index.getDimension(dimensionName);
|
||||
final IncrementalIndex.DimDim dimValLookup = index.getDimension(dimension);
|
||||
if (dimValLookup == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final int maxId = dimValLookup.size();
|
||||
final int dimIndex = index.getDimensionIndex(dimensionName);
|
||||
final int dimIndex = index.getDimensionIndex(dimension);
|
||||
|
||||
return new DimensionSelector()
|
||||
{
|
||||
|
@ -340,8 +339,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||
if (metricIndexInt == null) {
|
||||
return new FloatColumnSelector()
|
||||
{
|
||||
|
@ -372,9 +370,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public LongColumnSelector makeLongColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
|
||||
if(metricName.equals(Column.TIME_COLUMN_NAME)){
|
||||
if(columnName.equals(Column.TIME_COLUMN_NAME)){
|
||||
return new LongColumnSelector()
|
||||
{
|
||||
@Override
|
||||
|
@ -384,7 +380,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
};
|
||||
}
|
||||
final Integer metricIndexInt = index.getMetricIndex(metricName);
|
||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||
if (metricIndexInt == null) {
|
||||
return new LongColumnSelector()
|
||||
{
|
||||
|
@ -415,13 +411,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||
{
|
||||
final String columnName = column.toLowerCase();
|
||||
final Integer metricIndexInt = index.getMetricIndex(columnName);
|
||||
final Integer metricIndexInt = index.getMetricIndex(column);
|
||||
|
||||
if (metricIndexInt != null) {
|
||||
final int metricIndex = metricIndexInt;
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(index.getMetricType(columnName));
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(index.getMetricType(column));
|
||||
final BufferAggregator agg = index.getAggregator(metricIndex);
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
|
@ -442,7 +437,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
};
|
||||
}
|
||||
|
||||
final Integer dimensionIndexInt = index.getDimensionIndex(columnName);
|
||||
final Integer dimensionIndexInt = index.getDimensionIndex(column);
|
||||
|
||||
if (dimensionIndexInt != null) {
|
||||
final int dimensionIndex = dimensionIndexInt;
|
||||
|
@ -528,11 +523,11 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(String dimension, String value)
|
||||
{
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase());
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
||||
if (dimIndexObject == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension.toLowerCase());
|
||||
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension);
|
||||
if (!dimDim.contains(value)) {
|
||||
if (value == null || "".equals(value)) {
|
||||
final int dimIndex = dimIndexObject;
|
||||
|
@ -579,7 +574,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(String dimension, final Predicate<String> predicate)
|
||||
{
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase());
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
||||
if (dimIndexObject == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
@ -608,7 +603,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final Bound bound)
|
||||
{
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension.toLowerCase());
|
||||
Integer dimIndexObject = index.getDimensionIndex(dimension);
|
||||
if (dimIndexObject == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
|
|
@ -85,18 +85,7 @@ public class SpatialDimensionRowTransformer implements Function<InputRow, InputR
|
|||
// remove all spatial dimensions
|
||||
final List<String> finalDims = Lists.newArrayList(
|
||||
Iterables.filter(
|
||||
Lists.transform(
|
||||
row.getDimensions(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
)
|
||||
,
|
||||
row.getDimensions(),
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -82,7 +82,7 @@ public class QueryRunnerTestHelper
|
|||
);
|
||||
public static final QueryGranularity dayGran = QueryGranularity.DAY;
|
||||
public static final QueryGranularity allGran = QueryGranularity.ALL;
|
||||
public static final String marketDimension = "marKet";
|
||||
public static final String marketDimension = "market";
|
||||
public static final String qualityDimension = "quality";
|
||||
public static final String placementDimension = "placement";
|
||||
public static final String placementishDimension = "placementish";
|
||||
|
|
|
@ -965,7 +965,7 @@ public class GroupByQueryRunnerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithMixedCasingOrdering()
|
||||
public void testGroupByWithSameCaseOrdering()
|
||||
{
|
||||
GroupByQuery query = new GroupByQuery.Builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
|
@ -974,7 +974,7 @@ public class GroupByQueryRunnerTest
|
|||
Arrays.<DimensionSpec>asList(
|
||||
new DefaultDimensionSpec(
|
||||
QueryRunnerTestHelper.marketDimension,
|
||||
"MarketAlias"
|
||||
"marketalias"
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -983,7 +983,7 @@ public class GroupByQueryRunnerTest
|
|||
new DefaultLimitSpec(
|
||||
Lists.newArrayList(
|
||||
new OrderByColumnSpec(
|
||||
"marketALIAS",
|
||||
"marketalias",
|
||||
OrderByColumnSpec.Direction.DESCENDING
|
||||
)
|
||||
), 3
|
||||
|
@ -1058,7 +1058,13 @@ public class GroupByQueryRunnerTest
|
|||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "upfront", "rows", 186L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "total_market", "rows", 186L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"1970-01-01T00:00:00.000Z",
|
||||
"market",
|
||||
"total_market",
|
||||
"rows",
|
||||
186L
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "market", "spot", "rows", 837L)
|
||||
);
|
||||
|
||||
|
|
|
@ -72,7 +72,7 @@ public class SegmentAnalyzerTest
|
|||
Assert.assertEquals(TestIndex.COLUMNS.length, columns.size()); // All columns including time
|
||||
|
||||
for (String dimension : TestIndex.DIMENSIONS) {
|
||||
final ColumnAnalysis columnAnalysis = columns.get(dimension.toLowerCase());
|
||||
final ColumnAnalysis columnAnalysis = columns.get(dimension);
|
||||
|
||||
Assert.assertEquals(dimension, ValueType.STRING.name(), columnAnalysis.getType());
|
||||
Assert.assertTrue(dimension, columnAnalysis.getSize() > 0);
|
||||
|
@ -80,7 +80,7 @@ public class SegmentAnalyzerTest
|
|||
}
|
||||
|
||||
for (String metric : TestIndex.METRICS) {
|
||||
final ColumnAnalysis columnAnalysis = columns.get(metric.toLowerCase());
|
||||
final ColumnAnalysis columnAnalysis = columns.get(metric);
|
||||
|
||||
Assert.assertEquals(metric, ValueType.FLOAT.name(), columnAnalysis.getType());
|
||||
Assert.assertTrue(metric, columnAnalysis.getSize() > 0);
|
||||
|
|
|
@ -46,7 +46,7 @@ public class IndexMergerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPersistCaseInsensitive() throws Exception
|
||||
public void testPersist() throws Exception
|
||||
{
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
@ -66,7 +66,7 @@ public class IndexMergerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPersistMergeCaseInsensitive() throws Exception
|
||||
public void testPersistMerge() throws Exception
|
||||
{
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
|
||||
|
@ -76,16 +76,16 @@ public class IndexMergerTest
|
|||
toPersist2.add(
|
||||
new MapBasedInputRow(
|
||||
timestamp,
|
||||
Arrays.asList("DIm1", "DIM2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2", "DIm1", "10000", "DIM2", "100000000")
|
||||
Arrays.asList("dim1", "dim2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2")
|
||||
)
|
||||
);
|
||||
|
||||
toPersist2.add(
|
||||
new MapBasedInputRow(
|
||||
timestamp,
|
||||
Arrays.asList("dIM1", "dIm2"),
|
||||
ImmutableMap.<String, Object>of("DIm1", "1", "DIM2", "2", "dim1", "5", "dim2", "6")
|
||||
Arrays.asList("dim1", "dim2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "5", "dim2", "6")
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -122,7 +122,7 @@ public class IndexMergerTest
|
|||
FileUtils.deleteQuietly(tempDir2);
|
||||
FileUtils.deleteQuietly(mergedDir);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPersistEmptyColumn() throws Exception
|
||||
|
|
|
@ -57,14 +57,14 @@ public class TestIndex
|
|||
public static final String[] COLUMNS = new String[]{
|
||||
"ts",
|
||||
"market",
|
||||
"quALIty",
|
||||
"plAcEmEnT",
|
||||
"pLacementish",
|
||||
"iNdEx",
|
||||
"qualiTy_Uniques"
|
||||
"quality",
|
||||
"placement",
|
||||
"placementish",
|
||||
"index",
|
||||
"quality_uniques"
|
||||
};
|
||||
public static final String[] DIMENSIONS = new String[]{"market", "quALIty", "plAcEmEnT", "pLacementish"};
|
||||
public static final String[] METRICS = new String[]{"iNdEx"};
|
||||
public static final String[] DIMENSIONS = new String[]{"market", "quality", "placement", "placementish"};
|
||||
public static final String[] METRICS = new String[]{"index"};
|
||||
private static final Logger log = new Logger(TestIndex.class);
|
||||
private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z");
|
||||
private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
|
||||
|
|
|
@ -42,7 +42,7 @@ import java.util.Random;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
//@RunWith(Parameterized.class)
|
||||
public class BitmapCreationBenchmark extends AbstractBenchmark
|
||||
{
|
||||
private static final Logger log = new Logger(BitmapCreationBenchmark.class);
|
||||
|
@ -121,7 +121,7 @@ public class BitmapCreationBenchmark extends AbstractBenchmark
|
|||
|
||||
|
||||
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
|
||||
@Test
|
||||
//@Test
|
||||
public void testLinearAddition()
|
||||
{
|
||||
MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap();
|
||||
|
@ -132,7 +132,7 @@ public class BitmapCreationBenchmark extends AbstractBenchmark
|
|||
}
|
||||
|
||||
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 10)
|
||||
@Test
|
||||
//@Test
|
||||
public void testRandomAddition()
|
||||
{
|
||||
MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap();
|
||||
|
@ -143,7 +143,7 @@ public class BitmapCreationBenchmark extends AbstractBenchmark
|
|||
}
|
||||
|
||||
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
|
||||
@Test
|
||||
//@Test
|
||||
public void testLinearAdditionDescending()
|
||||
{
|
||||
MutableBitmap mutableBitmap = factory.makeEmptyMutableBitmap();
|
||||
|
@ -155,7 +155,7 @@ public class BitmapCreationBenchmark extends AbstractBenchmark
|
|||
|
||||
|
||||
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
|
||||
@Test
|
||||
//@Test
|
||||
public void testToImmutableByteArray()
|
||||
{
|
||||
ImmutableBitmap immutableBitmap = factory.makeImmutableBitmap(baseMutableBitmap);
|
||||
|
@ -164,7 +164,7 @@ public class BitmapCreationBenchmark extends AbstractBenchmark
|
|||
|
||||
|
||||
@BenchmarkOptions(warmupRounds = 10, benchmarkRounds = 1000)
|
||||
@Test
|
||||
//@Test
|
||||
public void testFromImmutableByteArray()
|
||||
{
|
||||
ImmutableBitmap immutableBitmap = factory.mapImmutableBitmap(baseByteBuffer);
|
||||
|
|
|
@ -54,16 +54,16 @@ public class IncrementalIndexTest
|
|||
index.add(
|
||||
new MapBasedInputRow(
|
||||
timestamp,
|
||||
Arrays.asList("Dim1", "DiM2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2", "DIM1", "3", "dIM2", "4")
|
||||
Arrays.asList("dim1", "dim2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "1", "dim2", "2")
|
||||
)
|
||||
);
|
||||
|
||||
index.add(
|
||||
new MapBasedInputRow(
|
||||
timestamp,
|
||||
Arrays.asList("diM1", "dIM2"),
|
||||
ImmutableMap.<String, Object>of("Dim1", "1", "DiM2", "2", "dim1", "3", "dim2", "4")
|
||||
Arrays.asList("dim1", "dim2"),
|
||||
ImmutableMap.<String, Object>of("dim1", "3", "dim2", "4")
|
||||
)
|
||||
);
|
||||
return index;
|
||||
|
@ -82,7 +82,7 @@ public class IncrementalIndexTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCaseInsensitivity() throws Exception
|
||||
public void testCaseSensitivity() throws Exception
|
||||
{
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
|
||||
|
|
|
@ -128,7 +128,7 @@ public class DatasourcesResource
|
|||
@QueryParam("full") final String full
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName);
|
||||
if (dataSource == null) {
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
@ -197,7 +197,7 @@ public class DatasourcesResource
|
|||
@QueryParam("full") String full
|
||||
)
|
||||
{
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName);
|
||||
|
||||
if (dataSource == null) {
|
||||
return Response.noContent().build();
|
||||
|
@ -258,7 +258,7 @@ public class DatasourcesResource
|
|||
@QueryParam("full") String full
|
||||
)
|
||||
{
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName);
|
||||
final Interval theInterval = new Interval(interval.replace("_", "/"));
|
||||
|
||||
if (dataSource == null) {
|
||||
|
@ -323,7 +323,7 @@ public class DatasourcesResource
|
|||
@QueryParam("full") String full
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName);
|
||||
if (dataSource == null) {
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
@ -356,7 +356,7 @@ public class DatasourcesResource
|
|||
@PathParam("segmentId") String segmentId
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName);
|
||||
if (dataSource == null) {
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue