mirror of https://github.com/apache/druid.git
somehow tests are passing
This commit is contained in:
parent
3cfa081c6c
commit
71269d7e88
|
@ -33,7 +33,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
@JsonSubTypes.Type(name="extraction", value=ExtractionDimFilter.class),
|
||||
@JsonSubTypes.Type(name="regex", value=RegexDimFilter.class),
|
||||
@JsonSubTypes.Type(name="search", value=SearchQueryDimFilter.class),
|
||||
@JsonSubTypes.Type(name="javascript", value=JavaScriptDimFilter.class)
|
||||
@JsonSubTypes.Type(name="javascript", value=JavaScriptDimFilter.class),
|
||||
@JsonSubTypes.Type(name="spatial", value=SpatialDimFilter.class)
|
||||
})
|
||||
public interface DimFilter
|
||||
{
|
||||
|
|
|
@ -35,6 +35,7 @@ class DimFilterCacheHelper
|
|||
static final byte REGEX_CACHE_ID = 0x5;
|
||||
static final byte SEARCH_QUERY_TYPE_ID = 0x6;
|
||||
static final byte JAVASCRIPT_CACHE_ID = 0x7;
|
||||
static final byte SPATIAL_CACHE_ID = 0x8;
|
||||
|
||||
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* 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.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.spatial.rtree.search.Bound;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SpatialDimFilter implements DimFilter
|
||||
{
|
||||
private final String dimension;
|
||||
private final Bound bound;
|
||||
|
||||
@JsonCreator
|
||||
public SpatialDimFilter(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("bound") Bound bound
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||
Preconditions.checkArgument(dimension.endsWith(".geo"), "must filter over geo dimension!");
|
||||
Preconditions.checkArgument(bound != null, "bound must not be null");
|
||||
|
||||
this.dimension = dimension;
|
||||
this.bound = bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] boundBytes = bound.getCacheKey();
|
||||
|
||||
return ByteBuffer.allocate(1 + boundBytes.length)
|
||||
.put(DimFilterCacheHelper.SPATIAL_CACHE_ID)
|
||||
.put(boundBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Bound getBound()
|
||||
{
|
||||
return bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SpatialDimFilter that = (SpatialDimFilter) o;
|
||||
|
||||
if (bound != null ? !bound.equals(that.bound) : that.bound != null) {
|
||||
return false;
|
||||
}
|
||||
if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = dimension != null ? dimension.hashCode() : 0;
|
||||
result = 31 * result + (bound != null ? bound.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -29,6 +29,12 @@ public abstract class AbstractColumn implements Column
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SpatialIndex getSpatialIndex()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn getComplexColumn()
|
||||
{
|
||||
|
|
|
@ -28,4 +28,5 @@ public interface BitmapIndex
|
|||
public int getCardinality();
|
||||
public String getValue(int index);
|
||||
public ImmutableConciseSet getConciseSet(String value);
|
||||
public ImmutableConciseSet getConciseSet(int idx);
|
||||
}
|
||||
|
|
|
@ -31,4 +31,5 @@ public interface Column
|
|||
public GenericColumn getGenericColumn();
|
||||
public ComplexColumn getComplexColumn();
|
||||
public BitmapIndex getBitmapIndex();
|
||||
public SpatialIndex getSpatialIndex();
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ public class ColumnBuilder
|
|||
private Supplier<GenericColumn> genericColumn = null;
|
||||
private Supplier<ComplexColumn> complexColumn = null;
|
||||
private Supplier<BitmapIndex> bitmapIndex = null;
|
||||
private Supplier<SpatialIndex> spatialIndex = null;
|
||||
|
||||
public ColumnBuilder setType(ValueType type)
|
||||
{
|
||||
|
@ -77,6 +78,12 @@ public class ColumnBuilder
|
|||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setSpatialIndex(Supplier<SpatialIndex> spatialIndex)
|
||||
{
|
||||
this.spatialIndex = spatialIndex;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Column build()
|
||||
{
|
||||
Preconditions.checkState(type != null, "Type must be set.");
|
||||
|
@ -86,6 +93,7 @@ public class ColumnBuilder
|
|||
.setType(type)
|
||||
.setDictionaryEncoded(dictionaryEncodedColumn != null)
|
||||
.setHasBitmapIndexes(bitmapIndex != null)
|
||||
.setHasSpatialIndexes(spatialIndex != null)
|
||||
.setRunLengthEncoded(runLengthColumn != null)
|
||||
.setHasMultipleValues(hasMultipleValues)
|
||||
,
|
||||
|
@ -93,7 +101,8 @@ public class ColumnBuilder
|
|||
runLengthColumn,
|
||||
genericColumn,
|
||||
complexColumn,
|
||||
bitmapIndex
|
||||
bitmapIndex,
|
||||
spatialIndex
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,5 +28,6 @@ public interface ColumnCapabilities
|
|||
public boolean isDictionaryEncoded();
|
||||
public boolean isRunLengthEncoded();
|
||||
public boolean hasBitmapIndexes();
|
||||
public boolean hasSpatialIndexes();
|
||||
public boolean hasMultipleValues();
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
private boolean dictionaryEncoded = false;
|
||||
private boolean runLengthEncoded = false;
|
||||
private boolean hasInvertedIndexes = false;
|
||||
private boolean hasSpatialIndexes = false;
|
||||
private boolean hasMultipleValues = false;
|
||||
|
||||
@Override
|
||||
|
@ -83,6 +84,19 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty("hasSpatialIndexes")
|
||||
public boolean hasSpatialIndexes()
|
||||
{
|
||||
return hasSpatialIndexes;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasSpatialIndexes(boolean hasSpatialIndexes)
|
||||
{
|
||||
this.hasSpatialIndexes = hasSpatialIndexes;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty("hasMultipleValues")
|
||||
public boolean hasMultipleValues()
|
||||
|
|
|
@ -32,6 +32,7 @@ class SimpleColumn implements Column
|
|||
private final Supplier<GenericColumn> genericColumn;
|
||||
private final Supplier<ComplexColumn> complexColumn;
|
||||
private final Supplier<BitmapIndex> bitmapIndex;
|
||||
private final Supplier<SpatialIndex> spatialIndex;
|
||||
|
||||
SimpleColumn(
|
||||
ColumnCapabilities capabilities,
|
||||
|
@ -39,7 +40,8 @@ class SimpleColumn implements Column
|
|||
Supplier<RunLengthColumn> runLengthColumn,
|
||||
Supplier<GenericColumn> genericColumn,
|
||||
Supplier<ComplexColumn> complexColumn,
|
||||
Supplier<BitmapIndex> bitmapIndex
|
||||
Supplier<BitmapIndex> bitmapIndex,
|
||||
Supplier<SpatialIndex> spatialIndex
|
||||
)
|
||||
{
|
||||
this.capabilities = capabilities;
|
||||
|
@ -48,6 +50,7 @@ class SimpleColumn implements Column
|
|||
this.genericColumn = genericColumn;
|
||||
this.complexColumn = complexColumn;
|
||||
this.bitmapIndex = bitmapIndex;
|
||||
this.spatialIndex = spatialIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,4 +101,10 @@ class SimpleColumn implements Column
|
|||
{
|
||||
return bitmapIndex == null ? null : bitmapIndex.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SpatialIndex getSpatialIndex()
|
||||
{
|
||||
return spatialIndex == null ? null : spatialIndex.get();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* 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.column;
|
||||
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface SpatialIndex
|
||||
{
|
||||
public int getCardinality();
|
||||
public String getValue(int index);
|
||||
public ImmutableRTree getRTree(String value);
|
||||
}
|
|
@ -63,11 +63,17 @@ public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex>
|
|||
{
|
||||
final int index = dictionary.indexOf(value);
|
||||
|
||||
if (index < 0) {
|
||||
return getConciseSet(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getConciseSet(int idx)
|
||||
{
|
||||
if (idx < 0) {
|
||||
return EMPTY_SET;
|
||||
}
|
||||
|
||||
final ImmutableConciseSet bitmap = bitmaps.get(index);
|
||||
final ImmutableConciseSet bitmap = bitmaps.get(idx);
|
||||
return bitmap == null ? EMPTY_SET : bitmap;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -22,10 +22,12 @@ package com.metamx.druid.index.serde;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedRTree;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
@ -35,7 +37,7 @@ import java.nio.ByteBuffer;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
*/
|
||||
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
|
@ -48,6 +50,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
private final VSizeIndexedInts singleValuedColumn;
|
||||
private final VSizeIndexed multiValuedColumn;
|
||||
private final GenericIndexed<ImmutableConciseSet> bitmaps;
|
||||
private final GenericIndexed<ImmutableRTree> spatialIndex;
|
||||
|
||||
private final int size;
|
||||
|
||||
|
@ -55,25 +58,26 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
GenericIndexed<String> dictionary,
|
||||
VSizeIndexedInts singleValCol,
|
||||
VSizeIndexed multiValCol,
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps,
|
||||
GenericIndexed<ImmutableRTree> spatialIndex
|
||||
)
|
||||
{
|
||||
this.dictionary = dictionary;
|
||||
this.singleValuedColumn = singleValCol;
|
||||
this.multiValuedColumn = multiValCol;
|
||||
this.bitmaps = bitmaps;
|
||||
this.spatialIndex = spatialIndex;
|
||||
|
||||
int size = dictionary.getSerializedSize();
|
||||
if (singleValCol != null && multiValCol == null) {
|
||||
size += singleValCol.getSerializedSize();
|
||||
}
|
||||
else if (singleValCol == null && multiValCol != null) {
|
||||
} else if (singleValCol == null && multiValCol != null) {
|
||||
size += multiValCol.getSerializedSize();
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
throw new IAE("Either singleValCol[%s] or multiValCol[%s] must be set", singleValCol, multiValCol);
|
||||
}
|
||||
size += bitmaps.getSerializedSize();
|
||||
size += spatialIndex.getSerializedSize();
|
||||
|
||||
this.size = size;
|
||||
}
|
||||
|
@ -84,6 +88,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
singleValuedColumn = null;
|
||||
multiValuedColumn = null;
|
||||
bitmaps = null;
|
||||
spatialIndex = null;
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
@ -106,11 +111,11 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
dictionary.writeToChannel(channel);
|
||||
if (isSingleValued()) {
|
||||
singleValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
multiValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
bitmaps.writeToChannel(channel);
|
||||
spatialIndex.writeToChannel(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -128,8 +133,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
multiValuedColumn = null;
|
||||
builder.setHasMultipleValues(false)
|
||||
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null));
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
singleValuedColumn = null;
|
||||
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
|
||||
builder.setHasMultipleValues(true)
|
||||
|
@ -140,8 +144,19 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
buffer, ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
|
||||
builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary));
|
||||
GenericIndexed<ImmutableRTree> spatialIndex = GenericIndexed.read(
|
||||
buffer, IndexedRTree.objectStrategy
|
||||
);
|
||||
|
||||
return new DictionaryEncodedColumnPartSerde(dictionary, singleValuedColumn, multiValuedColumn, bitmaps);
|
||||
builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary));
|
||||
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(spatialIndex, dictionary));
|
||||
|
||||
return new DictionaryEncodedColumnPartSerde(
|
||||
dictionary,
|
||||
singleValuedColumn,
|
||||
multiValuedColumn,
|
||||
bitmaps,
|
||||
spatialIndex
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.index.column.SpatialIndex;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SpatialIndexColumnPartSupplier implements Supplier<SpatialIndex>
|
||||
{
|
||||
private static final ImmutableRTree EMPTY_SET = new ImmutableRTree();
|
||||
|
||||
private final GenericIndexed<ImmutableRTree> indexedTree;
|
||||
private final GenericIndexed<String> dictionary;
|
||||
|
||||
public SpatialIndexColumnPartSupplier(
|
||||
GenericIndexed<ImmutableRTree> indexedTree,
|
||||
GenericIndexed<String> dictionary
|
||||
) {
|
||||
this.indexedTree = indexedTree;
|
||||
this.dictionary = dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SpatialIndex get()
|
||||
{
|
||||
return new SpatialIndex()
|
||||
{
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return dictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getValue(int index)
|
||||
{
|
||||
return dictionary.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getRTree(String value)
|
||||
{
|
||||
final int index = dictionary.indexOf(value);
|
||||
|
||||
if (index < 0) {
|
||||
return EMPTY_SET;
|
||||
}
|
||||
|
||||
final ImmutableRTree rTree = indexedTree.get(index);
|
||||
return rTree == null ? EMPTY_SET : rTree;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid.index.v1;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.BiMap;
|
||||
import com.google.common.collect.HashBiMap;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -64,6 +65,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
public class IncrementalIndex implements Iterable<Row>
|
||||
{
|
||||
private static final Logger log = new Logger(IncrementalIndex.class);
|
||||
private static final Joiner JOINER = Joiner.on(",");
|
||||
|
||||
private final long minTimestamp;
|
||||
private final QueryGranularity gran;
|
||||
|
@ -134,6 +136,12 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
List<String[]> overflow = null;
|
||||
for (String dimension : rowDimensions) {
|
||||
dimension = dimension.toLowerCase();
|
||||
List<String> dimensionValues = row.getDimension(dimension);
|
||||
|
||||
// FIXME: HACK!!! Rewrite this when this file is rewritten
|
||||
if (dimension.endsWith(".geo")) {
|
||||
dimensionValues = Arrays.asList(JOINER.join(dimensionValues));
|
||||
}
|
||||
|
||||
final Integer index = dimensionOrder.get(dimension);
|
||||
if (index == null) {
|
||||
|
@ -143,9 +151,9 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
if (overflow == null) {
|
||||
overflow = Lists.newArrayList();
|
||||
}
|
||||
overflow.add(getDimVals(dimValues.add(dimension), row.getDimension(dimension)));
|
||||
overflow.add(getDimVals(dimValues.add(dimension), dimensionValues));
|
||||
} else {
|
||||
dims[index] = getDimVals(dimValues.get(dimension), row.getDimension(dimension));
|
||||
dims[index] = getDimVals(dimValues.get(dimension), dimensionValues);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -409,8 +417,7 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
if (holder == null) {
|
||||
holder = new DimDim();
|
||||
dimensions.put(dimension, holder);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
throw new ISE("dimension[%s] already existed even though add() was called!?", dimension);
|
||||
}
|
||||
return holder;
|
||||
|
|
|
@ -20,9 +20,11 @@
|
|||
package com.metamx.druid.index.v1;
|
||||
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -45,6 +47,7 @@ public class Index
|
|||
final Map<String, Map<String, Integer>> dimIdLookup;
|
||||
final Map<String, String[]> reverseDimLookup;
|
||||
final Map<String, ImmutableConciseSet[]> indexes;
|
||||
final Map<String, ImmutableRTree> spatialIndexes;
|
||||
final Map<String, DimensionColumn> dimensionValues;
|
||||
|
||||
/*
|
||||
|
@ -79,6 +82,7 @@ public class Index
|
|||
Map<String, Map<String, Integer>> dimIdLookup,
|
||||
Map<String, String[]> reverseDimLookup,
|
||||
Map<String, ImmutableConciseSet[]> indexes,
|
||||
Map<String, ImmutableRTree> spatialIndexes,
|
||||
Map<String, DimensionColumn> dimensionValues
|
||||
)
|
||||
{
|
||||
|
@ -90,6 +94,7 @@ public class Index
|
|||
this.dimIdLookup = dimIdLookup;
|
||||
this.reverseDimLookup = reverseDimLookup;
|
||||
this.indexes = indexes;
|
||||
this.spatialIndexes = spatialIndexes;
|
||||
this.dimensionValues = dimensionValues;
|
||||
|
||||
for (int i = 0; i < dimensions.length; i++) {
|
||||
|
@ -126,4 +131,37 @@ public class Index
|
|||
return emptySet;
|
||||
}
|
||||
}
|
||||
|
||||
public ImmutableConciseSet getInvertedIndex(String dimension, int valueIndex)
|
||||
{
|
||||
try {
|
||||
return indexes.get(dimension)[valueIndex];
|
||||
}
|
||||
catch (NullPointerException e) {
|
||||
log.warn(
|
||||
e,
|
||||
"NPE on dimension[%s], valueIndex[%d], with index over interval[%s]",
|
||||
dimension,
|
||||
valueIndex,
|
||||
dataInterval
|
||||
);
|
||||
return emptySet;
|
||||
}
|
||||
}
|
||||
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
try {
|
||||
return spatialIndexes.get(dimension);
|
||||
}
|
||||
catch (NullPointerException e) {
|
||||
log.warn(
|
||||
e,
|
||||
"NPE on dimension[%s] over interval[%s]",
|
||||
dimension,
|
||||
dataInterval
|
||||
);
|
||||
return new ImmutableRTree();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import com.metamx.common.io.smoosh.Smoosh;
|
|||
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||
import com.metamx.common.io.smoosh.SmooshedWriter;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.index.QueryableIndex;
|
||||
import com.metamx.druid.index.SimpleQueryableIndex;
|
||||
import com.metamx.druid.index.column.Column;
|
||||
|
@ -53,11 +54,13 @@ import com.metamx.druid.index.serde.FloatGenericColumnPartSerde;
|
|||
import com.metamx.druid.index.serde.FloatGenericColumnSupplier;
|
||||
import com.metamx.druid.index.serde.LongGenericColumnPartSerde;
|
||||
import com.metamx.druid.index.serde.LongGenericColumnSupplier;
|
||||
import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.kv.ArrayIndexed;
|
||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedIterable;
|
||||
import com.metamx.druid.kv.IndexedRTree;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
import com.metamx.druid.utils.SerializerUtils;
|
||||
|
@ -221,7 +224,10 @@ public class IndexIO
|
|||
case 6:
|
||||
case 7:
|
||||
log.info("Old version, re-persisting.");
|
||||
IndexMerger.append(Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), converted);
|
||||
IndexMerger.append(
|
||||
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
|
||||
converted
|
||||
);
|
||||
return true;
|
||||
case 8:
|
||||
DefaultIndexIOHandler.convertV8toV9(toConvert, converted);
|
||||
|
@ -251,6 +257,7 @@ public class IndexIO
|
|||
* @return
|
||||
*/
|
||||
public boolean canBeMapped(File inDir) throws IOException;
|
||||
|
||||
public MMappedIndex mapDir(File inDir) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -265,6 +272,7 @@ public class IndexIO
|
|||
public static class DefaultIndexIOHandler implements IndexIOHandler
|
||||
{
|
||||
private static final Logger log = new Logger(DefaultIndexIOHandler.class);
|
||||
|
||||
@Override
|
||||
public Index readIndex(File inDir)
|
||||
{
|
||||
|
@ -325,6 +333,7 @@ public class IndexIO
|
|||
Map<String, GenericIndexed<String>> dimValueLookups = Maps.newHashMap();
|
||||
Map<String, VSizeIndexed> dimColumns = Maps.newHashMap();
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexed = Maps.newHashMap();
|
||||
Map<String, GenericIndexed<ImmutableRTree>> spatialIndexed = Maps.newHashMap();
|
||||
|
||||
for (String dimension : IndexedIterable.create(availableDimensions)) {
|
||||
ByteBuffer dimBuffer = smooshedFiles.mapFile(makeDimFile(inDir, dimension).getName());
|
||||
|
@ -348,6 +357,14 @@ public class IndexIO
|
|||
);
|
||||
}
|
||||
|
||||
ByteBuffer spatialBuffer = smooshedFiles.mapFile("spatial.drd");
|
||||
for (String dimension : IndexedIterable.create(availableDimensions)) {
|
||||
spatialIndexed.put(
|
||||
serializerUtils.readString(spatialBuffer),
|
||||
GenericIndexed.read(spatialBuffer, IndexedRTree.objectStrategy)
|
||||
);
|
||||
}
|
||||
|
||||
final MMappedIndex retVal = new MMappedIndex(
|
||||
availableDimensions,
|
||||
availableMetrics,
|
||||
|
@ -356,7 +373,8 @@ public class IndexIO
|
|||
metrics,
|
||||
dimValueLookups,
|
||||
dimColumns,
|
||||
invertedIndexed
|
||||
invertedIndexed,
|
||||
spatialIndexed
|
||||
);
|
||||
|
||||
log.debug("Mapped v8 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
|
||||
|
@ -402,6 +420,15 @@ public class IndexIO
|
|||
);
|
||||
}
|
||||
|
||||
Map<String, GenericIndexed<ImmutableRTree>> spatialIndexes = Maps.newHashMap();
|
||||
final ByteBuffer spatialBuffer = v8SmooshedFiles.mapFile("spatial.drd");
|
||||
while (spatialBuffer.hasRemaining()) {
|
||||
spatialIndexes.put(
|
||||
serializerUtils.readString(spatialBuffer),
|
||||
GenericIndexed.read(spatialBuffer, IndexedRTree.objectStrategy)
|
||||
);
|
||||
}
|
||||
|
||||
final LinkedHashSet<String> skippedFiles = Sets.newLinkedHashSet();
|
||||
final Set<String> skippedDimensions = Sets.newLinkedHashSet();
|
||||
for (String filename : v8SmooshedFiles.getInternalFilenames()) {
|
||||
|
@ -435,6 +462,7 @@ public class IndexIO
|
|||
VSizeIndexedInts singleValCol = null;
|
||||
VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer());
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps = bitmapIndexes.get(dimension);
|
||||
GenericIndexed<ImmutableRTree> spatialIndex = spatialIndexes.get(dimension);
|
||||
|
||||
boolean onlyOneValue = true;
|
||||
ConciseSet nullsSet = null;
|
||||
|
@ -476,8 +504,7 @@ public class IndexIO
|
|||
Iterables.concat(Arrays.asList(theNullSet), bitmaps),
|
||||
ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
bumpedDictionary = false;
|
||||
bitmaps = GenericIndexed.fromIterable(
|
||||
Iterables.concat(
|
||||
|
@ -487,8 +514,7 @@ public class IndexIO
|
|||
ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
}
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
bumpedDictionary = false;
|
||||
}
|
||||
|
||||
|
@ -517,7 +543,7 @@ public class IndexIO
|
|||
}
|
||||
|
||||
builder.addSerde(
|
||||
new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmaps)
|
||||
new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmaps, spatialIndex)
|
||||
);
|
||||
|
||||
final ColumnDescriptor serdeficator = builder.build();
|
||||
|
@ -673,8 +699,11 @@ public class IndexIO
|
|||
new BitmapIndexColumnPartSupplier(
|
||||
index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension)
|
||||
)
|
||||
).setSpatialIndex(
|
||||
new SpatialIndexColumnPartSupplier(
|
||||
index.getSpatialIndexes().get(dimension), index.getDimValueLookup(dimension)
|
||||
)
|
||||
.build()
|
||||
).build()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ package com.metamx.druid.index.v1;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -39,6 +41,9 @@ import com.metamx.common.guava.MergeIterable;
|
|||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.common.io.smoosh.Smoosh;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.common.spatial.rtree.RTree;
|
||||
import com.metamx.common.spatial.rtree.split.LinearGutmanSplitStrategy;
|
||||
import com.metamx.druid.CombiningIterable;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.aggregation.ToLowerCaseAggregatorFactory;
|
||||
|
@ -54,6 +59,7 @@ import com.metamx.druid.kv.IOPeon;
|
|||
import com.metamx.druid.kv.Indexed;
|
||||
import com.metamx.druid.kv.IndexedInts;
|
||||
import com.metamx.druid.kv.IndexedIterable;
|
||||
import com.metamx.druid.kv.IndexedRTree;
|
||||
import com.metamx.druid.kv.TmpFileIOPeon;
|
||||
import com.metamx.druid.kv.VSizeIndexedWriter;
|
||||
import com.metamx.druid.utils.JodaUtils;
|
||||
|
@ -88,6 +94,7 @@ public class IndexMerger
|
|||
|
||||
private static final SerializerUtils serializerUtils = new SerializerUtils();
|
||||
private static final int INVALID_ROW = -1;
|
||||
private static final Splitter SPLITTER = Splitter.on(",");
|
||||
|
||||
public static File persist(final IncrementalIndex index, File outDir) throws IOException
|
||||
{
|
||||
|
@ -682,6 +689,59 @@ public class IndexMerger
|
|||
System.currentTimeMillis() - startTime
|
||||
);
|
||||
|
||||
|
||||
/************ Create Geographical Indexes *************/
|
||||
// FIXME: Rewrite when indexing is updated
|
||||
Stopwatch stopwatch = new Stopwatch();
|
||||
stopwatch.start();
|
||||
|
||||
final File geoFile = new File(v8OutDir, "spatial.drd");
|
||||
Files.touch(geoFile);
|
||||
out = Files.newOutputStreamSupplier(geoFile, true);
|
||||
|
||||
for (int i = 0; i < mergedDimensions.size(); ++i) {
|
||||
String dimension = mergedDimensions.get(i);
|
||||
|
||||
File dimOutFile = dimOuts.get(i).getFile();
|
||||
final MappedByteBuffer dimValsMapped = Files.map(dimOutFile);
|
||||
|
||||
if (!dimension.equals(serializerUtils.readString(dimValsMapped))) {
|
||||
throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimension);
|
||||
}
|
||||
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy);
|
||||
log.info("Indexing geo dimension[%s] with cardinality[%,d]", dimension, dimVals.size());
|
||||
|
||||
FlattenedArrayWriter<ImmutableRTree> writer = new FlattenedArrayWriter<ImmutableRTree>(
|
||||
ioPeon, dimension, IndexedRTree.objectStrategy
|
||||
);
|
||||
writer.open();
|
||||
|
||||
RTree tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50));
|
||||
|
||||
if (dimension.endsWith(".geo")) {
|
||||
int count = 0;
|
||||
for (String dimVal : IndexedIterable.create(dimVals)) {
|
||||
progress.progress();
|
||||
|
||||
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
|
||||
float[] coords = new float[stringCoords.size()];
|
||||
for (int j = 0; j < coords.length; j++) {
|
||||
coords[j] = Float.valueOf(stringCoords.get(j));
|
||||
}
|
||||
tree.insert(coords, count);
|
||||
}
|
||||
}
|
||||
writer.write(ImmutableRTree.newImmutableFromMutable(tree));
|
||||
writer.close();
|
||||
|
||||
serializerUtils.writeString(out, dimension);
|
||||
ByteStreams.copy(writer.combineStreams(), out);
|
||||
ioPeon.cleanup();
|
||||
|
||||
log.info("Completed spatial dimension[%s] in %,d millis.", dimension, stopwatch.elapsedMillis());
|
||||
}
|
||||
|
||||
|
||||
/************ Create Inverted Indexes *************/
|
||||
startTime = System.currentTimeMillis();
|
||||
|
||||
|
@ -718,7 +778,10 @@ public class IndexMerger
|
|||
}
|
||||
|
||||
ConciseSet bitset = new ConciseSet();
|
||||
for (Integer row : CombiningIterable.createSplatted(convertedInverteds, Ordering.<Integer>natural().nullsFirst())) {
|
||||
for (Integer row : CombiningIterable.createSplatted(
|
||||
convertedInverteds,
|
||||
Ordering.<Integer>natural().nullsFirst()
|
||||
)) {
|
||||
if (row != INVALID_ROW) {
|
||||
bitset.add(row);
|
||||
}
|
||||
|
@ -734,12 +797,13 @@ public class IndexMerger
|
|||
|
||||
log.info("Completed dimension[%s] in %,d millis.", dimension, System.currentTimeMillis() - dimStartTime);
|
||||
}
|
||||
|
||||
log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime);
|
||||
|
||||
final ArrayList<String> expectedFiles = Lists.newArrayList(
|
||||
Iterables.concat(
|
||||
Arrays.asList(
|
||||
"index.drd", "inverted.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER)
|
||||
"index.drd", "inverted.drd", "spatial.drd", String.format("time_%s.drd", IndexIO.BYTE_ORDER)
|
||||
),
|
||||
Iterables.transform(mergedDimensions, GuavaUtils.formatFunction("dim_%s.drd")),
|
||||
Iterables.transform(
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
|
@ -54,6 +55,7 @@ public class MMappedIndex
|
|||
final Map<String, GenericIndexed<String>> dimValueLookups;
|
||||
final Map<String, VSizeIndexed> dimColumns;
|
||||
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes;
|
||||
final Map<String, GenericIndexed<ImmutableRTree>> spatialIndexes;
|
||||
|
||||
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
|
||||
|
||||
|
@ -65,7 +67,8 @@ public class MMappedIndex
|
|||
Map<String, MetricHolder> metrics,
|
||||
Map<String, GenericIndexed<String>> dimValueLookups,
|
||||
Map<String, VSizeIndexed> dimColumns,
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes,
|
||||
Map<String, GenericIndexed<ImmutableRTree>> spatialIndexes
|
||||
)
|
||||
{
|
||||
this.availableDimensions = availableDimensions;
|
||||
|
@ -76,6 +79,7 @@ public class MMappedIndex
|
|||
this.dimValueLookups = dimValueLookups;
|
||||
this.dimColumns = dimColumns;
|
||||
this.invertedIndexes = invertedIndexes;
|
||||
this.spatialIndexes = spatialIndexes;
|
||||
|
||||
for (int i = 0; i < availableMetrics.size(); i++) {
|
||||
metricIndexes.put(availableMetrics.get(i), i);
|
||||
|
@ -143,6 +147,11 @@ public class MMappedIndex
|
|||
return invertedIndexes;
|
||||
}
|
||||
|
||||
public Map<String, GenericIndexed<ImmutableRTree>> getSpatialIndexes()
|
||||
{
|
||||
return spatialIndexes;
|
||||
}
|
||||
|
||||
public ImmutableConciseSet getInvertedIndex(String dimension, String value)
|
||||
{
|
||||
final GenericIndexed<String> lookup = dimValueLookups.get(dimension);
|
||||
|
@ -159,6 +168,7 @@ public class MMappedIndex
|
|||
return (retVal == null) ? emptySet : retVal;
|
||||
}
|
||||
|
||||
/*
|
||||
public static MMappedIndex fromIndex(Index index)
|
||||
{
|
||||
log.info("Converting timestamps");
|
||||
|
@ -177,6 +187,8 @@ public class MMappedIndex
|
|||
|
||||
Map<String, VSizeIndexed> dimColumns = Maps.newHashMap();
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes = Maps.newLinkedHashMap();
|
||||
Map<String, GenericIndexed<ImmutableRTree>> spatialIndexes = Maps.newLinkedHashMap();
|
||||
|
||||
for (String dimension : Arrays.asList(index.dimensions)) {
|
||||
final String[] dimVals = index.reverseDimLookup.get(dimension);
|
||||
final DimensionColumn dimColumn = index.dimensionValues.get(dimension);
|
||||
|
@ -258,7 +270,9 @@ public class MMappedIndex
|
|||
index.metricVals,
|
||||
dimValueLookups,
|
||||
dimColumns,
|
||||
invertedIndexes
|
||||
invertedIndexes,
|
||||
spatialIndexes
|
||||
);
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
|
|
@ -0,0 +1,84 @@
|
|||
package com.metamx.druid.kv;
|
||||
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedRTree implements Comparable<IndexedRTree>
|
||||
{
|
||||
public static ObjectStrategy<ImmutableRTree> objectStrategy =
|
||||
new ImmutableRTreeObjectStrategy();
|
||||
|
||||
private static Ordering<ImmutableRTree> comparator = new Ordering<ImmutableRTree>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ImmutableRTree tree, ImmutableRTree tree1
|
||||
)
|
||||
{
|
||||
if (tree.size() == 0 && tree1.size() == 0) {
|
||||
return 0;
|
||||
}
|
||||
if (tree.size() == 0) {
|
||||
return -1;
|
||||
}
|
||||
if (tree1.size() == 0) {
|
||||
return 1;
|
||||
}
|
||||
return tree.compareTo(tree1);
|
||||
}
|
||||
}.nullsFirst();
|
||||
|
||||
private final ImmutableRTree immutableRTree;
|
||||
|
||||
public IndexedRTree(ImmutableRTree immutableRTree)
|
||||
{
|
||||
this.immutableRTree = immutableRTree;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(IndexedRTree spatialIndexedInts)
|
||||
{
|
||||
return immutableRTree.compareTo(spatialIndexedInts.getImmutableRTree());
|
||||
}
|
||||
|
||||
public ImmutableRTree getImmutableRTree()
|
||||
{
|
||||
return immutableRTree;
|
||||
}
|
||||
|
||||
private static class ImmutableRTreeObjectStrategy
|
||||
implements ObjectStrategy<ImmutableRTree>
|
||||
{
|
||||
@Override
|
||||
public Class<? extends ImmutableRTree> getClazz()
|
||||
{
|
||||
return ImmutableRTree.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes)
|
||||
{
|
||||
buffer.limit(buffer.position() + numBytes);
|
||||
return new ImmutableRTree(buffer.asReadOnlyBuffer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] toBytes(ImmutableRTree val)
|
||||
{
|
||||
if (val == null || val.size() == 0) {
|
||||
return new byte[]{};
|
||||
}
|
||||
return val.toBytes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(ImmutableRTree o1, ImmutableRTree o2)
|
||||
{
|
||||
return comparator.compare(o1, o2);
|
||||
}
|
||||
}
|
||||
}
|
2
pom.xml
2
pom.xml
|
@ -38,7 +38,7 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.21.0</metamx.java-util.version>
|
||||
<metamx.java-util.version>0.21.1-SNAPSHOT</metamx.java-util.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
|
|
|
@ -44,6 +44,8 @@ public abstract class BaseStorageAdapter implements StorageAdapter
|
|||
|
||||
public abstract ImmutableConciseSet getInvertedIndex(String dimension, String dimVal);
|
||||
|
||||
public abstract ImmutableConciseSet getInvertedIndex(String dimension, int idx);
|
||||
|
||||
public abstract Offset getFilterOffset(Filter filter);
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package com.metamx.druid.index.brita;
|
||||
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
||||
|
@ -29,4 +30,6 @@ public interface BitmapIndexSelector
|
|||
public Indexed<String> getDimensionValues(String dimension);
|
||||
public int getNumRows();
|
||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value);
|
||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx);
|
||||
public ImmutableRTree getSpatialIndex(String dimension);
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import com.metamx.druid.query.filter.OrDimFilter;
|
|||
import com.metamx.druid.query.filter.RegexDimFilter;
|
||||
import com.metamx.druid.query.filter.SearchQueryDimFilter;
|
||||
import com.metamx.druid.query.filter.SelectorDimFilter;
|
||||
import com.metamx.druid.query.filter.SpatialDimFilter;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
@ -89,6 +90,10 @@ public class Filters
|
|||
final JavaScriptDimFilter javaScriptDimFilter = (JavaScriptDimFilter) dimFilter;
|
||||
|
||||
filter = new JavaScriptFilter(javaScriptDimFilter.getDimension(), javaScriptDimFilter.getFunction());
|
||||
} else if (dimFilter instanceof SpatialDimFilter) {
|
||||
final SpatialDimFilter spatialDimFilter = (SpatialDimFilter) dimFilter;
|
||||
|
||||
filter = new SpatialFilter(spatialDimFilter.getDimension(), spatialDimFilter.getBound());
|
||||
}
|
||||
|
||||
return filter;
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.brita;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.spatial.rtree.search.Bound;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SpatialFilter implements Filter
|
||||
{
|
||||
private final String dimension;
|
||||
private final Bound bound;
|
||||
|
||||
public SpatialFilter(
|
||||
String dimension,
|
||||
Bound bound
|
||||
)
|
||||
{
|
||||
this.dimension = dimension;
|
||||
this.bound = bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector)
|
||||
{
|
||||
Iterable<Integer> indexes = selector.getSpatialIndex(dimension).search(bound);
|
||||
return ImmutableConciseSet.union(
|
||||
Iterables.transform(
|
||||
indexes,
|
||||
new Function<Integer, ImmutableConciseSet>()
|
||||
{
|
||||
@Override
|
||||
public ImmutableConciseSet apply(Integer input)
|
||||
{
|
||||
return selector.getConciseInvertedIndex(dimension, input);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||
{
|
||||
return factory.makeValueMatcher(
|
||||
dimension,
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
|
@ -28,6 +28,7 @@ import com.metamx.common.Pair;
|
|||
import com.metamx.common.collect.MoreIterators;
|
||||
import com.metamx.common.guava.FunctionalIterator;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.BaseStorageAdapter;
|
||||
import com.metamx.druid.Capabilities;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
|
@ -349,6 +350,12 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
|||
return index.getInvertedIndex(dimension.toLowerCase(), dimVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getInvertedIndex(String dimension, int idx)
|
||||
{
|
||||
return index.getInvertedIndex(dimension.toLowerCase(), idx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Offset getFilterOffset(Filter filter)
|
||||
{
|
||||
|
@ -519,5 +526,17 @@ public class IndexStorageAdapter extends BaseStorageAdapter
|
|||
{
|
||||
return index.getInvertedIndex(dimension.toLowerCase(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx)
|
||||
{
|
||||
return index.getInvertedIndex(dimension.toLowerCase(), idx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
return index.getSpatialIndex(dimension.toLowerCase());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.io.Closeables;
|
|||
import com.metamx.common.collect.MoreIterators;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.guava.FunctionalIterator;
|
||||
import com.metamx.common.spatial.rtree.ImmutableRTree;
|
||||
import com.metamx.druid.BaseStorageAdapter;
|
||||
import com.metamx.druid.Capabilities;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
|
@ -52,6 +53,7 @@ import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -221,6 +223,30 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
return column.getBitmapIndex().getConciseSet(dimVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getInvertedIndex(String dimension, int idx)
|
||||
{
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
if (column == null) {
|
||||
return new ImmutableConciseSet();
|
||||
}
|
||||
if (!column.getCapabilities().hasBitmapIndexes()) {
|
||||
return new ImmutableConciseSet();
|
||||
}
|
||||
|
||||
return column.getBitmapIndex().getConciseSet(idx);
|
||||
}
|
||||
|
||||
public ImmutableRTree getRTreeSpatialIndex(String dimension)
|
||||
{
|
||||
final Column column = index.getColumn(dimension.toLowerCase());
|
||||
if (column == null || !column.getCapabilities().hasSpatialIndexes()) {
|
||||
return new ImmutableRTree();
|
||||
}
|
||||
|
||||
return column.getSpatialIndex().getRTree(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Offset getFilterOffset(Filter filter)
|
||||
{
|
||||
|
@ -344,8 +370,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
return column.lookupId(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
|
@ -404,8 +429,8 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
if (cachedMetricVals == null) {
|
||||
Column holder = index.getColumn(metricName);
|
||||
if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -648,8 +673,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
return column.lookupId(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
|
@ -708,8 +732,8 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
if (cachedMetricVals == null) {
|
||||
Column holder = index.getColumn(metricName);
|
||||
if (holder != null && holder.getCapabilities().getType() == ValueType.FLOAT) {
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
cachedMetricVals = holder.getGenericColumn();
|
||||
genericColumnCache.put(metricName, cachedMetricVals);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -863,5 +887,17 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter
|
|||
{
|
||||
return getInvertedIndex(dimension, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx)
|
||||
{
|
||||
return getInvertedIndex(dimension, idx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
return getRTreeSpatialIndex(dimension);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue