Merge branch 'druid-0.7.x' of github.com:metamx/druid into druid-0.7.x

This commit is contained in:
fjy 2014-11-11 17:30:43 -08:00
commit 2f498c3422
63 changed files with 1224 additions and 461 deletions

View File

@ -21,6 +21,7 @@ package io.druid.common.utils;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import com.google.common.io.OutputSupplier; import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints;
import io.druid.collections.IntList; import io.druid.collections.IntList;
import java.io.IOException; import java.io.IOException;
@ -262,4 +263,9 @@ public class SerializerUtils
return retVal; return retVal;
} }
public int getSerializedStringByteSize(String str)
{
return Ints.BYTES + str.getBytes().length;
}
} }

View File

@ -29,7 +29,13 @@ public class MetadataStorageConnectorConfig
private boolean createTables = true; private boolean createTables = true;
@JsonProperty @JsonProperty
private String connectURI = null; private String host = "localhost";
@JsonProperty
private int port = 1527;
@JsonProperty
private String connectURI;
@JsonProperty @JsonProperty
private String user = null; private String user = null;
@ -37,16 +43,26 @@ public class MetadataStorageConnectorConfig
@JsonProperty @JsonProperty
private String password = null; private String password = null;
@JsonProperty
private String validationQuery = "SELECT 1";
public boolean isCreateTables() public boolean isCreateTables()
{ {
return createTables; return createTables;
} }
public String getHost()
{
return host;
}
public int getPort()
{
return port;
}
public String getConnectURI() public String getConnectURI()
{ {
if (connectURI == null) {
return String.format("jdbc:derby://%s:%s/druid;create=true", host, port);
}
return connectURI; return connectURI;
} }
@ -60,20 +76,14 @@ public class MetadataStorageConnectorConfig
return password; return password;
} }
public String getValidationQuery()
{
return validationQuery;
}
@Override @Override
public String toString() public String toString()
{ {
return "DbConnectorConfig{" + return "DbConnectorConfig{" +
"createTables=" + createTables + "createTables=" + createTables +
", connectURI='" + connectURI + '\'' + ", connectURI='" + getConnectURI() + '\'' +
", user='" + user + '\'' + ", user='" + user + '\'' +
", password=****" + ", password=****" +
", validationQuery='" + validationQuery + '\'' +
'}'; '}';
} }
} }

View File

@ -27,7 +27,7 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import io.druid.segment.IndexIO; import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker; import io.druid.segment.IndexMerger;
import io.druid.segment.IndexableAdapter; import io.druid.segment.IndexableAdapter;
import io.druid.segment.QueryableIndexIndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter;
import io.druid.segment.Rowboat; import io.druid.segment.Rowboat;
@ -37,7 +37,6 @@ import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.VersionedIntervalTimeline;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -106,7 +105,7 @@ public class AppendTask extends MergeTaskBase
); );
} }
return IndexMaker.append(adapters, outDir); return IndexMerger.append(adapters, outDir);
} }
@Override @Override

View File

@ -28,7 +28,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexIO; import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker; import io.druid.segment.IndexMerger;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -60,7 +60,7 @@ public class MergeTask extends MergeTaskBase
public File merge(final Map<DataSegment, File> segments, final File outDir) public File merge(final Map<DataSegment, File> segments, final File outDir)
throws Exception throws Exception
{ {
return IndexMaker.mergeQueryableIndex( return IndexMerger.mergeQueryableIndex(
Lists.transform( Lists.transform(
ImmutableList.copyOf(segments.values()), ImmutableList.copyOf(segments.values()),
new Function<File, QueryableIndex>() new Function<File, QueryableIndex>()

View File

@ -72,7 +72,7 @@ public class MySQLConnector extends SQLMetadataConnector
} }
@Override @Override
protected boolean tableExists(Handle handle, String tableName) public boolean tableExists(Handle handle, String tableName)
{ {
return !handle.createQuery("SHOW tables LIKE :tableName") return !handle.createQuery("SHOW tables LIKE :tableName")
.bind("tableName", tableName) .bind("tableName", tableName)

View File

@ -92,7 +92,7 @@
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>bytebuffer-collections</artifactId> <artifactId>bytebuffer-collections</artifactId>
<version>0.0.4</version> <version>0.1.1</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
@ -194,11 +194,6 @@
<artifactId>curator-x-discovery</artifactId> <artifactId>curator-x-discovery</artifactId>
<version>${apache.curator.version}</version> <version>${apache.curator.version}</version>
</dependency> </dependency>
<dependency>
<groupId>it.uniroma3.mat</groupId>
<artifactId>extendedset</artifactId>
<version>1.3.7</version>
</dependency>
<dependency> <dependency>
<groupId>com.google.guava</groupId> <groupId>com.google.guava</groupId>
<artifactId>guava</artifactId> <artifactId>guava</artifactId>

View File

@ -65,7 +65,7 @@ public class PostgreSQLConnector extends SQLMetadataConnector
} }
@Override @Override
protected boolean tableExists(final Handle handle, final String tableName) public boolean tableExists(final Handle handle, final String tableName)
{ {
return !handle.createQuery( return !handle.createQuery(
"SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE :tableName" "SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE :tableName"

View File

@ -54,10 +54,6 @@
<groupId>org.skife.config</groupId> <groupId>org.skife.config</groupId>
<artifactId>config-magic</artifactId> <artifactId>config-magic</artifactId>
</dependency> </dependency>
<dependency>
<groupId>it.uniroma3.mat</groupId>
<artifactId>extendedset</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.slf4j</groupId> <groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId> <artifactId>slf4j-log4j12</artifactId>

View File

@ -19,9 +19,10 @@
package io.druid.query.filter; package io.druid.query.filter;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
/** /**
*/ */
@ -29,7 +30,8 @@ public interface BitmapIndexSelector
{ {
public Indexed<String> getDimensionValues(String dimension); public Indexed<String> getDimensionValues(String dimension);
public int getNumRows(); public int getNumRows();
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value); public BitmapFactory getBitmapFactory();
public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx); public ImmutableBitmap getBitmapIndex(String dimension, String value);
public ImmutableBitmap getBitmapIndex(String dimension, int idx);
public ImmutableRTree getSpatialIndex(String dimension); public ImmutableRTree getSpatialIndex(String dimension);
} }

View File

@ -19,12 +19,12 @@
package io.druid.query.filter; package io.druid.query.filter;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import com.metamx.collections.bitmap.ImmutableBitmap;
/** /**
*/ */
public interface Filter public interface Filter
{ {
public ImmutableConciseSet goConcise(BitmapIndexSelector selector); public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector);
public ValueMatcher makeMatcher(ValueMatcherFactory factory); public ValueMatcher makeMatcher(ValueMatcherFactory factory);
} }

View File

@ -122,7 +122,7 @@ public class SegmentAnalyzer
String value = bitmapIndex.getValue(i); String value = bitmapIndex.getValue(i);
if (value != null) { if (value != null) {
size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getConciseSet(value).size(); size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getBitmap(value).size();
} }
} }

View File

@ -21,9 +21,12 @@ package io.druid.query.search;
import com.google.common.base.Strings; import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterable;
@ -48,8 +51,8 @@ import io.druid.segment.column.BitmapIndex;
import io.druid.segment.column.Column; import io.druid.segment.column.Column;
import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.Filters; import io.druid.segment.filter.Filters;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.TreeSet; import java.util.TreeSet;
@ -94,12 +97,14 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
dimsToSearch = dimensions; dimsToSearch = dimensions;
} }
BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions();
final ImmutableConciseSet baseFilter; final ImmutableBitmap baseFilter;
if (filter == null) { if (filter == null) {
baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); baseFilter = bitmapFactory.complement(bitmapFactory.makeEmptyImmutableBitmap(), index.getNumRows());
} else { } else {
baseFilter = filter.goConcise(new ColumnSelectorBitmapIndexSelector(index)); ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(bitmapFactory, index);
baseFilter = filter.getBitmapIndex(selector);
} }
for (String dimension : dimsToSearch) { for (String dimension : dimsToSearch) {
@ -113,7 +118,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
for (int i = 0; i < bitmapIndex.getCardinality(); ++i) { for (int i = 0; i < bitmapIndex.getCardinality(); ++i) {
String dimVal = Strings.nullToEmpty(bitmapIndex.getValue(i)); String dimVal = Strings.nullToEmpty(bitmapIndex.getValue(i));
if (searchQuerySpec.accept(dimVal) && if (searchQuerySpec.accept(dimVal) &&
ImmutableConciseSet.intersection(baseFilter, bitmapIndex.getConciseSet(i)).size() > 0) { bitmapFactory.intersection(Arrays.asList(baseFilter, bitmapIndex.getBitmap(i))).size() > 0) {
retVal.add(new SearchHit(dimension, dimVal)); retVal.add(new SearchHit(dimension, dimVal));
if (retVal.size() >= limit) { if (retVal.size() >= limit) {
return makeReturnResult(limit, retVal); return makeReturnResult(limit, retVal);

View File

@ -19,30 +19,35 @@
package io.druid.segment; package io.druid.segment;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.segment.data.Offset; import io.druid.segment.data.Offset;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.roaringbitmap.IntIterator;
import it.uniroma3.mat.extendedset.intset.IntSet;
/** /**
*/ */
public class ConciseOffset implements Offset public class BitmapOffset implements Offset
{ {
private static final int INVALID_VALUE = -1; private static final int INVALID_VALUE = -1;
IntSet.IntIterator itr; private final IntIterator itr;
private final ImmutableConciseSet invertedIndex; private final BitmapFactory bitmapFactory;
private final ImmutableBitmap bitmapIndex;
private volatile int val; private volatile int val;
public ConciseOffset(ImmutableConciseSet invertedIndex) public BitmapOffset(BitmapFactory bitmapFactory, ImmutableBitmap bitmapIndex)
{ {
this.invertedIndex = invertedIndex; this.bitmapFactory = bitmapFactory;
this.itr = invertedIndex.iterator(); this.bitmapIndex = bitmapIndex;
this.itr = bitmapIndex.iterator();
increment(); increment();
} }
private ConciseOffset(ConciseOffset otherOffset) private BitmapOffset(BitmapOffset otherOffset)
{ {
this.invertedIndex = otherOffset.invertedIndex; this.bitmapFactory = otherOffset.bitmapFactory;
this.bitmapIndex = otherOffset.bitmapIndex;
this.itr = otherOffset.itr.clone(); this.itr = otherOffset.itr.clone();
this.val = otherOffset.val; this.val = otherOffset.val;
} }
@ -66,11 +71,11 @@ public class ConciseOffset implements Offset
@Override @Override
public Offset clone() public Offset clone()
{ {
if (invertedIndex == null || invertedIndex.size() == 0) { if (bitmapIndex == null || bitmapIndex.size() == 0) {
return new ConciseOffset(new ImmutableConciseSet()); return new BitmapOffset(bitmapFactory, bitmapFactory.makeEmptyImmutableBitmap());
} }
return new ConciseOffset(this); return new BitmapOffset(this);
} }
@Override @Override

View File

@ -19,6 +19,8 @@
package io.druid.segment; package io.druid.segment;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.CloseQuietly;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
@ -27,7 +29,6 @@ import io.druid.segment.column.DictionaryEncodedColumn;
import io.druid.segment.column.GenericColumn; import io.druid.segment.column.GenericColumn;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.IndexedIterable;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.Iterator; import java.util.Iterator;
@ -35,12 +36,15 @@ import java.util.Iterator;
*/ */
public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
{ {
private final BitmapFactory bitmapFactory;
private final ColumnSelector index; private final ColumnSelector index;
public ColumnSelectorBitmapIndexSelector( public ColumnSelectorBitmapIndexSelector(
final BitmapFactory bitmapFactory,
final ColumnSelector index final ColumnSelector index
) )
{ {
this.bitmapFactory = bitmapFactory;
this.index = index; this.index = index;
} }
@ -100,33 +104,40 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
} }
@Override @Override
public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value) public BitmapFactory getBitmapFactory()
{ {
final Column column = index.getColumn(dimension.toLowerCase()); return bitmapFactory;
if (column == null) {
return new ImmutableConciseSet();
}
if (!column.getCapabilities().hasBitmapIndexes()) {
return new ImmutableConciseSet();
}
return column.getBitmapIndex().getConciseSet(value);
} }
@Override @Override
public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx) public ImmutableBitmap getBitmapIndex(String dimension, String value)
{ {
final Column column = index.getColumn(dimension.toLowerCase()); final Column column = index.getColumn(dimension.toLowerCase());
if (column == null) { if (column == null) {
return new ImmutableConciseSet(); return bitmapFactory.makeEmptyImmutableBitmap();
} }
if (!column.getCapabilities().hasBitmapIndexes()) { if (!column.getCapabilities().hasBitmapIndexes()) {
return new ImmutableConciseSet(); bitmapFactory.makeEmptyImmutableBitmap();
} }
return column.getBitmapIndex().getBitmap(value);
}
@Override
public ImmutableBitmap getBitmapIndex(String dimension, int idx)
{
final Column column = index.getColumn(dimension.toLowerCase());
if (column == null || column.getCapabilities() == null) {
bitmapFactory.makeEmptyImmutableBitmap();
}
if (!column.getCapabilities().hasBitmapIndexes()) {
bitmapFactory.makeEmptyImmutableBitmap();
}
// This is a workaround given the current state of indexing, I feel shame // This is a workaround given the current state of indexing, I feel shame
final int index1 = column.getBitmapIndex().hasNulls() ? idx + 1 : idx; final int index1 = column.getBitmapIndex().hasNulls() ? idx + 1 : idx;
return column.getBitmapIndex().getConciseSet(index1); return column.getBitmapIndex().getBitmap(index1);
} }
@Override @Override

View File

@ -35,6 +35,10 @@ import com.google.common.primitives.Ints;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Injector; import com.google.inject.Injector;
import com.google.inject.Module; import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ConciseBitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import com.metamx.common.ISE; import com.metamx.common.ISE;
@ -47,7 +51,7 @@ import com.metamx.emitter.EmittingLogger;
import io.druid.common.utils.SerializerUtils; import io.druid.common.utils.SerializerUtils;
import io.druid.guice.ConfigProvider; import io.druid.guice.ConfigProvider;
import io.druid.guice.GuiceInjectors; import io.druid.guice.GuiceInjectors;
import io.druid.jackson.DefaultObjectMapper; import io.druid.guice.JsonConfigProvider;
import io.druid.query.DruidProcessingConfig; import io.druid.query.DruidProcessingConfig;
import io.druid.segment.column.Column; import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnBuilder;
@ -55,9 +59,10 @@ import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.segment.data.ArrayIndexed; import io.druid.segment.data.ArrayIndexed;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ByteBufferSerializer; import io.druid.segment.data.ByteBufferSerializer;
import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier;
import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.IndexedIterable;
import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.IndexedRTree;
@ -73,8 +78,6 @@ import io.druid.segment.serde.FloatGenericColumnSupplier;
import io.druid.segment.serde.LongGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde;
import io.druid.segment.serde.LongGenericColumnSupplier; import io.druid.segment.serde.LongGenericColumnSupplier;
import io.druid.segment.serde.SpatialIndexColumnPartSupplier; import io.druid.segment.serde.SpatialIndexColumnPartSupplier;
import it.uniroma3.mat.extendedset.intset.ConciseSet;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
@ -120,6 +123,8 @@ public class IndexIO
private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final SerializerUtils serializerUtils = new SerializerUtils();
private static final ObjectMapper mapper; private static final ObjectMapper mapper;
private static final BitmapSerdeFactory bitmapSerdeFactory;
protected static final ColumnConfig columnConfig; protected static final ColumnConfig columnConfig;
static { static {
@ -136,26 +141,19 @@ public class IndexIO
ImmutableMap.of("base_path", "druid.processing") ImmutableMap.of("base_path", "druid.processing")
); );
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
} }
} }
) )
); );
mapper = injector.getInstance(ObjectMapper.class); mapper = injector.getInstance(ObjectMapper.class);
columnConfig = injector.getInstance(ColumnConfig.class); columnConfig = injector.getInstance(ColumnConfig.class);
} bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class);
private static volatile IndexIOHandler handler = null;
@Deprecated
public static MMappedIndex mapDir(final File inDir) throws IOException
{
init();
return handler.mapDir(inDir);
} }
public static QueryableIndex loadIndex(File inDir) throws IOException public static QueryableIndex loadIndex(File inDir) throws IOException
{ {
init();
final int version = SegmentUtils.getVersionFromDir(inDir); final int version = SegmentUtils.getVersionFromDir(inDir);
final IndexLoader loader = indexLoaders.get(version); final IndexLoader loader = indexLoaders.get(version);
@ -167,27 +165,6 @@ public class IndexIO
} }
} }
public static boolean hasHandler()
{
return (IndexIO.handler != null);
}
public static void registerHandler(IndexIOHandler handler)
{
if (IndexIO.handler == null) {
IndexIO.handler = handler;
} else {
throw new ISE("Already have a handler[%s], cannot register another[%s]", IndexIO.handler, handler);
}
}
private static void init()
{
if (handler == null) {
handler = new DefaultIndexIOHandler();
}
}
public static int getVersionFromDir(File inDir) throws IOException public static int getVersionFromDir(File inDir) throws IOException
{ {
File versionFile = new File(inDir, "version.bin"); File versionFile = new File(inDir, "version.bin");
@ -228,7 +205,7 @@ public class IndexIO
case 6: case 6:
case 7: case 7:
log.info("Old version, re-persisting."); log.info("Old version, re-persisting.");
IndexMaker.append( IndexMerger.append(
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
converted converted
); );
@ -280,6 +257,7 @@ public class IndexIO
indexBuffer, GenericIndexed.stringStrategy indexBuffer, GenericIndexed.stringStrategy
); );
final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer));
final BitmapSerdeFactory conciseBitmapSerdeFactory = new ConciseBitmapSerdeFactory();
CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer( CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer(
smooshedFiles.mapFile(makeTimeFile(inDir, BYTE_ORDER).getName()), BYTE_ORDER smooshedFiles.mapFile(makeTimeFile(inDir, BYTE_ORDER).getName()), BYTE_ORDER
@ -298,7 +276,7 @@ public class IndexIO
Map<String, GenericIndexed<String>> dimValueLookups = Maps.newHashMap(); Map<String, GenericIndexed<String>> dimValueLookups = Maps.newHashMap();
Map<String, VSizeIndexed> dimColumns = Maps.newHashMap(); Map<String, VSizeIndexed> dimColumns = Maps.newHashMap();
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexed = Maps.newHashMap(); Map<String, GenericIndexed<ImmutableBitmap>> bitmaps = Maps.newHashMap();
for (String dimension : IndexedIterable.create(availableDimensions)) { for (String dimension : IndexedIterable.create(availableDimensions)) {
ByteBuffer dimBuffer = smooshedFiles.mapFile(makeDimFile(inDir, dimension).getName()); ByteBuffer dimBuffer = smooshedFiles.mapFile(makeDimFile(inDir, dimension).getName());
@ -316,9 +294,9 @@ public class IndexIO
ByteBuffer invertedBuffer = smooshedFiles.mapFile("inverted.drd"); ByteBuffer invertedBuffer = smooshedFiles.mapFile("inverted.drd");
for (int i = 0; i < availableDimensions.size(); ++i) { for (int i = 0; i < availableDimensions.size(); ++i) {
invertedIndexed.put( bitmaps.put(
serializerUtils.readString(invertedBuffer), serializerUtils.readString(invertedBuffer),
GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) GenericIndexed.read(invertedBuffer, conciseBitmapSerdeFactory.getObjectStrategy())
); );
} }
@ -327,7 +305,10 @@ public class IndexIO
while (spatialBuffer != null && spatialBuffer.hasRemaining()) { while (spatialBuffer != null && spatialBuffer.hasRemaining()) {
spatialIndexed.put( spatialIndexed.put(
serializerUtils.readString(spatialBuffer), serializerUtils.readString(spatialBuffer),
ByteBufferSerializer.read(spatialBuffer, IndexedRTree.objectStrategy) ByteBufferSerializer.read(
spatialBuffer,
new IndexedRTree.ImmutableRTreeObjectStrategy(conciseBitmapSerdeFactory.getBitmapFactory())
)
); );
} }
@ -339,7 +320,7 @@ public class IndexIO
metrics, metrics,
dimValueLookups, dimValueLookups,
dimColumns, dimColumns,
invertedIndexed, bitmaps,
spatialIndexed, spatialIndexed,
smooshedFiles smooshedFiles
); );
@ -371,13 +352,14 @@ public class IndexIO
final FileSmoosher v9Smoosher = new FileSmoosher(v9Dir); final FileSmoosher v9Smoosher = new FileSmoosher(v9Dir);
ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin"))); ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin")));
Map<String, GenericIndexed<ImmutableConciseSet>> bitmapIndexes = Maps.newHashMap();
Map<String, GenericIndexed<ImmutableBitmap>> bitmapIndexes = Maps.newHashMap();
final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd"); final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd");
while (invertedBuffer.hasRemaining()) { while (invertedBuffer.hasRemaining()) {
final String dimName = serializerUtils.readString(invertedBuffer);
bitmapIndexes.put( bitmapIndexes.put(
serializerUtils.readString(invertedBuffer), dimName,
GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy) GenericIndexed.read(invertedBuffer, bitmapSerdeFactory.getObjectStrategy())
); );
} }
@ -386,7 +368,11 @@ public class IndexIO
while (spatialBuffer != null && spatialBuffer.hasRemaining()) { while (spatialBuffer != null && spatialBuffer.hasRemaining()) {
spatialIndexes.put( spatialIndexes.put(
serializerUtils.readString(spatialBuffer), serializerUtils.readString(spatialBuffer),
ByteBufferSerializer.read(spatialBuffer, IndexedRTree.objectStrategy) ByteBufferSerializer.read(
spatialBuffer, new IndexedRTree.ImmutableRTreeObjectStrategy(
bitmapSerdeFactory.getBitmapFactory()
)
)
); );
} }
@ -422,11 +408,12 @@ public class IndexIO
VSizeIndexedInts singleValCol = null; VSizeIndexedInts singleValCol = null;
VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer());
GenericIndexed<ImmutableConciseSet> bitmaps = bitmapIndexes.get(dimension); GenericIndexed<ImmutableBitmap> bitmaps = bitmapIndexes.get(dimension);
ImmutableRTree spatialIndex = spatialIndexes.get(dimension); ImmutableRTree spatialIndex = spatialIndexes.get(dimension);
final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
boolean onlyOneValue = true; boolean onlyOneValue = true;
ConciseSet nullsSet = null; MutableBitmap nullsSet = null;
for (int i = 0; i < multiValCol.size(); ++i) { for (int i = 0; i < multiValCol.size(); ++i) {
VSizeIndexedInts rowValue = multiValCol.get(i); VSizeIndexedInts rowValue = multiValCol.get(i);
if (!onlyOneValue) { if (!onlyOneValue) {
@ -437,7 +424,7 @@ public class IndexIO
} }
if (rowValue.size() == 0) { if (rowValue.size() == 0) {
if (nullsSet == null) { if (nullsSet == null) {
nullsSet = new ConciseSet(); nullsSet = bitmapFactory.makeEmptyMutableBitmap();
} }
nullsSet.add(i); nullsSet.add(i);
} }
@ -448,7 +435,7 @@ public class IndexIO
final boolean bumpedDictionary; final boolean bumpedDictionary;
if (nullsSet != null) { if (nullsSet != null) {
log.info("Dimension[%s] has null rows.", dimension); log.info("Dimension[%s] has null rows.", dimension);
final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullsSet); final ImmutableBitmap theNullSet = bitmapFactory.makeImmutableBitmap(nullsSet);
if (dictionary.get(0) != null) { if (dictionary.get(0) != null) {
log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension); log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension);
@ -463,16 +450,19 @@ public class IndexIO
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat(Arrays.asList(theNullSet), bitmaps), Iterables.concat(Arrays.asList(theNullSet), bitmaps),
ConciseCompressedIndexedInts.objectStrategy bitmapSerdeFactory.getObjectStrategy()
); );
} else { } else {
bumpedDictionary = false; bumpedDictionary = false;
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat( Iterables.concat(
Arrays.asList(ImmutableConciseSet.union(theNullSet, bitmaps.get(0))), Arrays.asList(
bitmapFactory
.union(Arrays.asList(theNullSet, bitmaps.get(0)))
),
Iterables.skip(bitmaps, 1) Iterables.skip(bitmaps, 1)
), ),
ConciseCompressedIndexedInts.objectStrategy bitmapSerdeFactory.getObjectStrategy()
); );
} }
} else { } else {
@ -508,6 +498,7 @@ public class IndexIO
dictionary, dictionary,
singleValCol, singleValCol,
multiValCol, multiValCol,
bitmapSerdeFactory,
bitmaps, bitmaps,
spatialIndex spatialIndex
) )
@ -620,6 +611,10 @@ public class IndexIO
indexBuffer, GenericIndexed.stringStrategy indexBuffer, GenericIndexed.stringStrategy
); );
final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer));
final BitmapSerdeFactory segmentBitmapSerdeFactory = mapper.readValue(
serializerUtils.readString(indexBuffer),
BitmapSerdeFactory.class
);
Set<String> columns = Sets.newTreeSet(); Set<String> columns = Sets.newTreeSet();
columns.addAll(Lists.newArrayList(dims9)); columns.addAll(Lists.newArrayList(dims9));
@ -627,12 +622,17 @@ public class IndexIO
GenericIndexed<String> cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); GenericIndexed<String> cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy);
final long numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16; final String segmentBitmapSerdeFactoryString = mapper.writeValueAsString(segmentBitmapSerdeFactory);
final long numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16
+ serializerUtils.getSerializedStringByteSize(segmentBitmapSerdeFactoryString);
final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes);
cols.writeToChannel(writer); cols.writeToChannel(writer);
dims9.writeToChannel(writer); dims9.writeToChannel(writer);
serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getStartMillis());
serializerUtils.writeLong(writer, dataInterval.getEndMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis());
serializerUtils.writeString(writer, segmentBitmapSerdeFactoryString);
writer.close(); writer.close();
log.info("Skipped files[%s]", skippedFiles); log.info("Skipped files[%s]", skippedFiles);
@ -648,10 +648,12 @@ public class IndexIO
static class LegacyIndexLoader implements IndexLoader static class LegacyIndexLoader implements IndexLoader
{ {
private static final IndexIOHandler legacyHandler = new DefaultIndexIOHandler();
@Override @Override
public QueryableIndex load(File inDir) throws IOException public QueryableIndex load(File inDir) throws IOException
{ {
MMappedIndex index = IndexIO.mapDir(inDir); MMappedIndex index = legacyHandler.mapDir(inDir);
Map<String, Column> columns = Maps.newHashMap(); Map<String, Column> columns = Maps.newHashMap();
@ -669,7 +671,9 @@ public class IndexIO
) )
.setBitmapIndex( .setBitmapIndex(
new BitmapIndexColumnPartSupplier( new BitmapIndexColumnPartSupplier(
index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension) new ConciseBitmapFactory(),
index.getBitmapIndexes().get(dimension),
index.getDimValueLookup(dimension)
) )
); );
if (index.getSpatialIndexes().get(dimension) != null) { if (index.getSpatialIndexes().get(dimension) != null) {
@ -719,14 +723,17 @@ public class IndexIO
} }
String[] cols = colSet.toArray(new String[colSet.size()]); String[] cols = colSet.toArray(new String[colSet.size()]);
columns.put(Column.TIME_COLUMN_NAME, new ColumnBuilder() columns.put(
Column.TIME_COLUMN_NAME, new ColumnBuilder()
.setType(ValueType.LONG) .setType(ValueType.LONG)
.setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) .setGenericColumn(new LongGenericColumnSupplier(index.timestamps))
.build()); .build()
);
return new SimpleQueryableIndex( return new SimpleQueryableIndex(
index.getDataInterval(), index.getDataInterval(),
new ArrayIndexed<>(cols, String.class), new ArrayIndexed<>(cols, String.class),
index.getAvailableDimensions(), index.getAvailableDimensions(),
new ConciseBitmapFactory(),
columns, columns,
index.getFileMapper() index.getFileMapper()
); );
@ -752,6 +759,17 @@ public class IndexIO
final GenericIndexed<String> cols = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); final GenericIndexed<String> cols = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy);
final GenericIndexed<String> dims = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy); final GenericIndexed<String> dims = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy);
final Interval dataInterval = new Interval(indexBuffer.getLong(), indexBuffer.getLong()); final Interval dataInterval = new Interval(indexBuffer.getLong(), indexBuffer.getLong());
final BitmapSerdeFactory segmentBitmapSerdeFactory;
/**
* This is a workaround for the fact that in v8 segments, we have no information about the type of bitmap
* index to use. Since we cannot very cleanly build v9 segments directly, we are using a workaround where
* this information is appended to the end of index.drd.
*/
if (indexBuffer.hasRemaining()) {
segmentBitmapSerdeFactory = mapper.readValue(serializerUtils.readString(indexBuffer), BitmapSerdeFactory.class);
} else {
segmentBitmapSerdeFactory = BitmapSerdeFactory.DEFAULT_BITMAP_SERDE_FACTORY;
}
Map<String, Column> columns = Maps.newHashMap(); Map<String, Column> columns = Maps.newHashMap();
@ -762,7 +780,7 @@ public class IndexIO
columns.put(Column.TIME_COLUMN_NAME, deserializeColumn(mapper, smooshedFiles.mapFile("__time"))); columns.put(Column.TIME_COLUMN_NAME, deserializeColumn(mapper, smooshedFiles.mapFile("__time")));
final QueryableIndex index = new SimpleQueryableIndex( final QueryableIndex index = new SimpleQueryableIndex(
dataInterval, cols, dims, columns, smooshedFiles dataInterval, cols, dims, segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles
); );
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);

View File

@ -24,6 +24,7 @@ import com.google.common.base.Function;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.base.Splitter; import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -33,7 +34,12 @@ import com.google.common.collect.Sets;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.google.inject.Binder;
import com.google.inject.Injector; import com.google.inject.Injector;
import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.collections.spatial.RTree; import com.metamx.collections.spatial.RTree;
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
@ -46,21 +52,20 @@ import com.metamx.common.io.smoosh.FileSmoosher;
import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.io.smoosh.SmooshedWriter;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.collections.CombiningIterable; import io.druid.collections.CombiningIterable;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidPool;
import io.druid.common.utils.JodaUtils; import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils; import io.druid.common.utils.SerializerUtils;
import io.druid.guice.GuiceInjectors; import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedFloatsIndexedSupplier;
import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier;
import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseCompressedIndexedInts;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedInts;
@ -76,15 +81,12 @@ import io.druid.segment.serde.ComplexMetrics;
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde;
import io.druid.segment.serde.LongGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde;
import it.uniroma3.mat.extendedset.intset.ConciseSet;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -109,10 +111,23 @@ public class IndexMaker
private static final int INVALID_ROW = -1; private static final int INVALID_ROW = -1;
private static final Splitter SPLITTER = Splitter.on(","); private static final Splitter SPLITTER = Splitter.on(",");
private static final ObjectMapper mapper; private static final ObjectMapper mapper;
private static final BitmapSerdeFactory bitmapSerdeFactory;
static { static {
final Injector injector = GuiceInjectors.makeStartupInjector(); final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.<Module>of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
}
}
)
);
mapper = injector.getInstance(ObjectMapper.class); mapper = injector.getInstance(ObjectMapper.class);
bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class);
} }
public static File persist(final IncrementalIndex index, File outDir) throws IOException public static File persist(final IncrementalIndex index, File outDir) throws IOException
@ -170,7 +185,7 @@ public class IndexMaker
log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
return merge( return merge(
Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index)), Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index, bitmapSerdeFactory.getBitmapFactory())),
index.getMetricAggs(), index.getMetricAggs(),
outDir, outDir,
progress progress
@ -839,13 +854,14 @@ public class IndexMaker
? new MultiValColumnDictionaryEntryStore() ? new MultiValColumnDictionaryEntryStore()
: new SingleValColumnDictionaryEntryStore(); : new SingleValColumnDictionaryEntryStore();
ConciseSet nullSet = null; final BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
MutableBitmap nullSet = null;
int rowCount = 0; int rowCount = 0;
for (Rowboat theRow : theRows) { for (Rowboat theRow : theRows) {
if (dimIndex > theRow.getDims().length) { if (dimIndex > theRow.getDims().length) {
if (nullSet == null) { if (nullSet == null) {
nullSet = new ConciseSet(); nullSet = bitmapFactory.makeEmptyMutableBitmap();
} }
nullSet.add(rowCount); nullSet.add(rowCount);
adder.add(null); adder.add(null);
@ -853,7 +869,7 @@ public class IndexMaker
int[] dimVals = theRow.getDims()[dimIndex]; int[] dimVals = theRow.getDims()[dimIndex];
if (dimVals == null || dimVals.length == 0) { if (dimVals == null || dimVals.length == 0) {
if (nullSet == null) { if (nullSet == null) {
nullSet = new ConciseSet(); nullSet = bitmapFactory.makeEmptyMutableBitmap();
} }
nullSet.add(rowCount); nullSet.add(rowCount);
} }
@ -1062,18 +1078,18 @@ public class IndexMaker
} }
// Make bitmap indexes // Make bitmap indexes
List<ConciseSet> conciseSets = Lists.newArrayList(); List<MutableBitmap> mutableBitmaps = Lists.newArrayList();
for (String dimVal : dimensionValues) { for (String dimVal : dimensionValues) {
List<Iterable<Integer>> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); List<Iterable<Integer>> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size());
for (int j = 0; j < adapters.size(); ++j) { for (int j = 0; j < adapters.size(); ++j) {
convertedInverteds.add( convertedInverteds.add(
new ConvertingIndexedInts( new ConvertingIndexedInts(
adapters.get(j).getInverteds(dimension, dimVal), rowNumConversions.get(j) adapters.get(j).getBitmapIndex(dimension, dimVal), rowNumConversions.get(j)
) )
); );
} }
ConciseSet bitset = new ConciseSet(); MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap();
for (Integer row : CombiningIterable.createSplatted( for (Integer row : CombiningIterable.createSplatted(
convertedInverteds, convertedInverteds,
Ordering.<Integer>natural().nullsFirst() Ordering.<Integer>natural().nullsFirst()
@ -1083,40 +1099,40 @@ public class IndexMaker
} }
} }
conciseSets.add(bitset); mutableBitmaps.add(bitset);
} }
GenericIndexed<ImmutableConciseSet> bitmaps; GenericIndexed<ImmutableBitmap> bitmaps;
if (nullSet != null) { if (nullSet != null) {
final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullSet); final ImmutableBitmap theNullSet = bitmapFactory.makeImmutableBitmap(nullSet);
if (bumpDictionary) { if (bumpDictionary) {
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat( Iterables.concat(
Arrays.asList(theNullSet), Arrays.asList(theNullSet),
Iterables.transform( Iterables.transform(
conciseSets, mutableBitmaps,
new Function<ConciseSet, ImmutableConciseSet>() new Function<MutableBitmap, ImmutableBitmap>()
{ {
@Override @Override
public ImmutableConciseSet apply(ConciseSet input) public ImmutableBitmap apply(MutableBitmap input)
{ {
return ImmutableConciseSet.newImmutableFromMutable(input); return bitmapFactory.makeImmutableBitmap(input);
} }
} }
) )
), ),
ConciseCompressedIndexedInts.objectStrategy bitmapSerdeFactory.getObjectStrategy()
); );
} else { } else {
Iterable<ImmutableConciseSet> immutableConciseSets = Iterables.transform( Iterable<ImmutableBitmap> immutableBitmaps = Iterables.transform(
conciseSets, mutableBitmaps,
new Function<ConciseSet, ImmutableConciseSet>() new Function<MutableBitmap, ImmutableBitmap>()
{ {
@Override @Override
public ImmutableConciseSet apply(ConciseSet input) public ImmutableBitmap apply(MutableBitmap input)
{ {
return ImmutableConciseSet.newImmutableFromMutable(input); return bitmapFactory.makeImmutableBitmap(input);
} }
} }
); );
@ -1124,30 +1140,27 @@ public class IndexMaker
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat( Iterables.concat(
Arrays.asList( Arrays.asList(
ImmutableConciseSet.union( theNullSet.union(Iterables.getFirst(immutableBitmaps, null))
theNullSet,
Iterables.getFirst(immutableConciseSets, null)
)
), ),
Iterables.skip(immutableConciseSets, 1) Iterables.skip(immutableBitmaps, 1)
), ),
ConciseCompressedIndexedInts.objectStrategy bitmapSerdeFactory.getObjectStrategy()
); );
} }
} else { } else {
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.transform( Iterables.transform(
conciseSets, mutableBitmaps,
new Function<ConciseSet, ImmutableConciseSet>() new Function<MutableBitmap, ImmutableBitmap>()
{ {
@Override @Override
public ImmutableConciseSet apply(ConciseSet input) public ImmutableBitmap apply(MutableBitmap input)
{ {
return ImmutableConciseSet.newImmutableFromMutable(input); return bitmapFactory.makeImmutableBitmap(input);
} }
} }
), ),
ConciseCompressedIndexedInts.objectStrategy bitmapSerdeFactory.getObjectStrategy()
); );
} }
@ -1156,7 +1169,11 @@ public class IndexMaker
boolean hasSpatialIndexes = columnCapabilities.get(dimension).hasSpatialIndexes(); boolean hasSpatialIndexes = columnCapabilities.get(dimension).hasSpatialIndexes();
RTree tree = null; RTree tree = null;
if (hasSpatialIndexes) { if (hasSpatialIndexes) {
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); tree = new RTree(
2,
new LinearGutmanSplitStrategy(0, 50, bitmapSerdeFactory.getBitmapFactory()),
bitmapSerdeFactory.getBitmapFactory()
);
} }
int dimValIndex = 0; int dimValIndex = 0;
@ -1168,7 +1185,7 @@ public class IndexMaker
for (int j = 0; j < coords.length; j++) { for (int j = 0; j < coords.length; j++) {
coords[j] = Float.valueOf(stringCoords.get(j)); coords[j] = Float.valueOf(stringCoords.get(j));
} }
tree.insert(coords, conciseSets.get(dimValIndex)); tree.insert(coords, mutableBitmaps.get(dimValIndex));
} }
dimValIndex++; dimValIndex++;
} }
@ -1185,6 +1202,7 @@ public class IndexMaker
dictionary, dictionary,
singleValCol, singleValCol,
multiValCol, multiValCol,
bitmapSerdeFactory,
bitmaps, bitmaps,
spatialIndex spatialIndex
), ),
@ -1354,7 +1372,12 @@ public class IndexMaker
GenericIndexed<String> cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.stringStrategy); GenericIndexed<String> cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.stringStrategy);
GenericIndexed<String> dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.stringStrategy); GenericIndexed<String> dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.stringStrategy);
final long numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; final String bitmapSerdeFactoryType = mapper.writeValueAsString(bitmapSerdeFactory);
final long numBytes = cols.getSerializedSize()
+ dims.getSerializedSize()
+ 16
+ serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType);
final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes);
cols.writeToChannel(writer); cols.writeToChannel(writer);
@ -1371,6 +1394,9 @@ public class IndexMaker
serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getStartMillis());
serializerUtils.writeLong(writer, dataInterval.getEndMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis());
serializerUtils.writeString(
writer, bitmapSerdeFactoryType
);
writer.close(); writer.close();
IndexIO.checkFileSize(new File(outDir, "index.drd")); IndexIO.checkFileSize(new File(outDir, "index.drd"));

View File

@ -19,9 +19,11 @@
package io.druid.segment; package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Splitter; import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -32,6 +34,12 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.common.io.OutputSupplier; import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.collections.spatial.RTree; import com.metamx.collections.spatial.RTree;
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
@ -48,15 +56,17 @@ import io.druid.common.guava.FileOutputSupplier;
import io.druid.common.guava.GuavaUtils; import io.druid.common.guava.GuavaUtils;
import io.druid.common.utils.JodaUtils; import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils; 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.AggregatorFactory;
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedLongsSupplierSerializer;
import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseCompressedIndexedInts;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.GenericIndexedWriter;
import io.druid.segment.data.IOPeon; import io.druid.segment.data.IOPeon;
@ -71,8 +81,6 @@ import io.druid.segment.incremental.IncrementalIndexAdapter;
import io.druid.segment.serde.ComplexMetricColumnSerializer; import io.druid.segment.serde.ComplexMetricColumnSerializer;
import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetricSerde;
import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.ComplexMetrics;
import it.uniroma3.mat.extendedset.intset.ConciseSet;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -103,6 +111,27 @@ public class IndexMerger
private static final int INVALID_ROW = -1; private static final int INVALID_ROW = -1;
private static final Splitter SPLITTER = Splitter.on(","); private static final Splitter SPLITTER = Splitter.on(",");
private static final ObjectMapper mapper;
private static final BitmapSerdeFactory bitmapSerdeFactory;
static {
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.<Module>of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
}
}
)
);
mapper = injector.getInstance(ObjectMapper.class);
bitmapSerdeFactory = injector.getInstance(BitmapSerdeFactory.class);
}
public static File persist(final IncrementalIndex index, File outDir) throws IOException public static File persist(final IncrementalIndex index, File outDir) throws IOException
{ {
return persist(index, index.getInterval(), outDir); return persist(index, index.getInterval(), outDir);
@ -153,7 +182,13 @@ public class IndexMerger
log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
return merge( return merge(
Arrays.<IndexableAdapter>asList(new IncrementalIndexAdapter(dataInterval, index)), Arrays.<IndexableAdapter>asList(
new IncrementalIndexAdapter(
dataInterval,
index,
bitmapSerdeFactory.getBitmapFactory()
)
),
index.getMetricAggs(), index.getMetricAggs(),
outDir, outDir,
progress progress
@ -458,6 +493,7 @@ public class IndexMerger
dataInterval = new Interval(minTime, maxTime); dataInterval = new Interval(minTime, maxTime);
serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime)); serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime));
serializerUtils.writeString(channel, mapper.writeValueAsString(bitmapSerdeFactory));
} }
finally { finally {
CloseQuietly.close(channel); CloseQuietly.close(channel);
@ -756,8 +792,8 @@ public class IndexMerger
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy); Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy);
log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); log.info("Starting dimension[%s] with cardinality[%,d]", dimension, dimVals.size());
GenericIndexedWriter<ImmutableConciseSet> writer = new GenericIndexedWriter<ImmutableConciseSet>( GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
ioPeon, dimension, ConciseCompressedIndexedInts.objectStrategy ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy()
); );
writer.open(); writer.open();
@ -766,11 +802,12 @@ public class IndexMerger
RTree tree = null; RTree tree = null;
IOPeon spatialIoPeon = new TmpFileIOPeon(); IOPeon spatialIoPeon = new TmpFileIOPeon();
if (isSpatialDim) { if (isSpatialDim) {
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
spatialWriter = new ByteBufferWriter<ImmutableRTree>( spatialWriter = new ByteBufferWriter<ImmutableRTree>(
spatialIoPeon, dimension, IndexedRTree.objectStrategy spatialIoPeon, dimension, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory)
); );
spatialWriter.open(); spatialWriter.open();
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory);
} }
for (String dimVal : IndexedIterable.create(dimVals)) { for (String dimVal : IndexedIterable.create(dimVals)) {
@ -779,12 +816,12 @@ public class IndexMerger
for (int j = 0; j < indexes.size(); ++j) { for (int j = 0; j < indexes.size(); ++j) {
convertedInverteds.add( convertedInverteds.add(
new ConvertingIndexedInts( new ConvertingIndexedInts(
indexes.get(j).getInverteds(dimension, dimVal), rowNumConversions.get(j) indexes.get(j).getBitmapIndex(dimension, dimVal), rowNumConversions.get(j)
) )
); );
} }
ConciseSet bitset = new ConciseSet(); MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap();
for (Integer row : CombiningIterable.createSplatted( for (Integer row : CombiningIterable.createSplatted(
convertedInverteds, convertedInverteds,
Ordering.<Integer>natural().nullsFirst() Ordering.<Integer>natural().nullsFirst()
@ -794,7 +831,9 @@ public class IndexMerger
} }
} }
writer.write(ImmutableConciseSet.newImmutableFromMutable(bitset)); writer.write(
bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset)
);
if (isSpatialDim && dimVal != null) { if (isSpatialDim && dimVal != null) {
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
@ -906,6 +945,9 @@ public class IndexMerger
serializerUtils.writeString( serializerUtils.writeString(
channel, String.format("%s/%s", dataInterval.getStart(), dataInterval.getEnd()) channel, String.format("%s/%s", dataInterval.getStart(), dataInterval.getEnd())
); );
serializerUtils.writeString(
channel, mapper.writeValueAsString(bitmapSerdeFactory)
);
} }
finally { finally {
CloseQuietly.close(channel); CloseQuietly.close(channel);

View File

@ -42,7 +42,7 @@ public interface IndexableAdapter
Iterable<Rowboat> getRows(); Iterable<Rowboat> getRows();
IndexedInts getInverteds(String dimension, String value); IndexedInts getBitmapIndex(String dimension, String value);
String getMetricType(String metric); String getMetricType(String metric);

View File

@ -19,13 +19,13 @@
package io.druid.segment; package io.druid.segment;
import com.google.common.collect.Maps; import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.io.smoosh.SmooshedFileMapper;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IndexedLongs;
import io.druid.segment.data.VSizeIndexed; import io.druid.segment.data.VSizeIndexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -38,7 +38,6 @@ import java.util.Map;
public class MMappedIndex public class MMappedIndex
{ {
private static final Logger log = new Logger(MMappedIndex.class); private static final Logger log = new Logger(MMappedIndex.class);
private static final ImmutableConciseSet emptySet = new ImmutableConciseSet();
final GenericIndexed<String> availableDimensions; final GenericIndexed<String> availableDimensions;
final GenericIndexed<String> availableMetrics; final GenericIndexed<String> availableMetrics;
@ -47,12 +46,10 @@ public class MMappedIndex
final Map<String, MetricHolder> metrics; final Map<String, MetricHolder> metrics;
final Map<String, GenericIndexed<String>> dimValueLookups; final Map<String, GenericIndexed<String>> dimValueLookups;
final Map<String, VSizeIndexed> dimColumns; final Map<String, VSizeIndexed> dimColumns;
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes; final Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes;
final Map<String, ImmutableRTree> spatialIndexes; final Map<String, ImmutableRTree> spatialIndexes;
final SmooshedFileMapper fileMapper; final SmooshedFileMapper fileMapper;
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
public MMappedIndex( public MMappedIndex(
GenericIndexed<String> availableDimensions, GenericIndexed<String> availableDimensions,
GenericIndexed<String> availableMetrics, GenericIndexed<String> availableMetrics,
@ -61,7 +58,7 @@ public class MMappedIndex
Map<String, MetricHolder> metrics, Map<String, MetricHolder> metrics,
Map<String, GenericIndexed<String>> dimValueLookups, Map<String, GenericIndexed<String>> dimValueLookups,
Map<String, VSizeIndexed> dimColumns, Map<String, VSizeIndexed> dimColumns,
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes, Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes,
Map<String, ImmutableRTree> spatialIndexes, Map<String, ImmutableRTree> spatialIndexes,
SmooshedFileMapper fileMapper SmooshedFileMapper fileMapper
) )
@ -76,10 +73,6 @@ public class MMappedIndex
this.invertedIndexes = invertedIndexes; this.invertedIndexes = invertedIndexes;
this.spatialIndexes = spatialIndexes; this.spatialIndexes = spatialIndexes;
this.fileMapper = fileMapper; this.fileMapper = fileMapper;
for (int i = 0; i < availableMetrics.size(); i++) {
metricIndexes.put(availableMetrics.get(i), i);
}
} }
public CompressedLongsIndexedSupplier getTimestamps() public CompressedLongsIndexedSupplier getTimestamps()
@ -102,21 +95,11 @@ public class MMappedIndex
return metrics; return metrics;
} }
public Integer getMetricIndex(String metricName)
{
return metricIndexes.get(metricName);
}
public Interval getDataInterval() public Interval getDataInterval()
{ {
return dataInterval; return dataInterval;
} }
public IndexedLongs getReadOnlyTimestamps()
{
return timestamps.get();
}
public MetricHolder getMetricHolder(String metric) public MetricHolder getMetricHolder(String metric)
{ {
final MetricHolder retVal = metrics.get(metric); final MetricHolder retVal = metrics.get(metric);
@ -138,7 +121,7 @@ public class MMappedIndex
return dimColumns.get(dimension); return dimColumns.get(dimension);
} }
public Map<String, GenericIndexed<ImmutableConciseSet>> getInvertedIndexes() public Map<String, GenericIndexed<ImmutableBitmap>> getBitmapIndexes()
{ {
return invertedIndexes; return invertedIndexes;
} }
@ -148,22 +131,6 @@ public class MMappedIndex
return spatialIndexes; return spatialIndexes;
} }
public ImmutableConciseSet getInvertedIndex(String dimension, String value)
{
final GenericIndexed<String> lookup = dimValueLookups.get(dimension);
if (lookup == null) {
return emptySet;
}
int indexOf = lookup.indexOf(value);
if (indexOf < 0) {
return emptySet;
}
ImmutableConciseSet retVal = invertedIndexes.get(dimension).get(indexOf);
return (retVal == null) ? emptySet : retVal;
}
public SmooshedFileMapper getFileMapper() public SmooshedFileMapper getFileMapper()
{ {
return fileMapper; return fileMapper;

View File

@ -17,7 +17,10 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.segment;import io.druid.segment.data.Indexed; package io.druid.segment;
import com.metamx.collections.bitmap.BitmapFactory;
import io.druid.segment.data.Indexed;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.io.IOException; import java.io.IOException;
@ -30,6 +33,7 @@ public interface QueryableIndex extends ColumnSelector
public int getNumRows(); public int getNumRows();
public Indexed<String> getColumnNames(); public Indexed<String> getColumnNames();
public Indexed<String> getAvailableDimensions(); public Indexed<String> getAvailableDimensions();
public BitmapFactory getBitmapFactoryForDimensions();
/** /**
* The close method shouldn't actually be here as this is nasty. We will adjust it in the future. * The close method shouldn't actually be here as this is nasty. We will adjust it in the future.

View File

@ -35,7 +35,7 @@ import io.druid.segment.column.IndexedFloatsGenericColumn;
import io.druid.segment.column.IndexedLongsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.ArrayBasedIndexedInts;
import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.BitmapCompressedIndexedInts;
import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.EmptyIndexedInts;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedInts;
@ -279,7 +279,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
} }
@Override @Override
public IndexedInts getInverteds(String dimension, String value) public IndexedInts getBitmapIndex(String dimension, String value)
{ {
final Column column = input.getColumn(dimension); final Column column = input.getColumn(dimension);
@ -292,7 +292,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
return new EmptyIndexedInts(); return new EmptyIndexedInts();
} }
return new ConciseCompressedIndexedInts(bitmaps.getConciseSet(value)); return new BitmapCompressedIndexedInts(bitmaps.getBitmap(value));
} }
@Override @Override

View File

@ -158,7 +158,11 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
if (filter == null) { if (filter == null) {
sequence = new NoFilterCursorSequenceBuilder(index, actualInterval, gran).build(); sequence = new NoFilterCursorSequenceBuilder(index, actualInterval, gran).build();
} else { } else {
Offset offset = new ConciseOffset(filter.goConcise(new ColumnSelectorBitmapIndexSelector(index))); final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
index.getBitmapFactoryForDimensions(),
index
);
final Offset offset = new BitmapOffset(selector.getBitmapFactory(), filter.getBitmapIndex(selector));
sequence = new CursorSequenceBuilder(index, actualInterval, gran, offset).build(); sequence = new CursorSequenceBuilder(index, actualInterval, gran, offset).build();
} }

View File

@ -76,9 +76,9 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
} }
@Override @Override
public IndexedInts getInverteds(String dimension, String value) public IndexedInts getBitmapIndex(String dimension, String value)
{ {
return baseAdapter.getInverteds(dimension, value); return baseAdapter.getBitmapIndex(dimension, value);
} }
@Override @Override

View File

@ -20,6 +20,7 @@
package io.druid.segment; package io.druid.segment;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.io.smoosh.SmooshedFileMapper;
import io.druid.segment.column.Column; import io.druid.segment.column.Column;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
@ -35,6 +36,7 @@ public class SimpleQueryableIndex implements QueryableIndex
private final Interval dataInterval; private final Interval dataInterval;
private final Indexed<String> columnNames; private final Indexed<String> columnNames;
private final Indexed<String> availableDimensions; private final Indexed<String> availableDimensions;
private final BitmapFactory bitmapFactory;
private final Map<String, Column> columns; private final Map<String, Column> columns;
private final SmooshedFileMapper fileMapper; private final SmooshedFileMapper fileMapper;
@ -42,6 +44,7 @@ public class SimpleQueryableIndex implements QueryableIndex
Interval dataInterval, Interval dataInterval,
Indexed<String> columnNames, Indexed<String> columnNames,
Indexed<String> dimNames, Indexed<String> dimNames,
BitmapFactory bitmapFactory,
Map<String, Column> columns, Map<String, Column> columns,
SmooshedFileMapper fileMapper SmooshedFileMapper fileMapper
) )
@ -50,6 +53,7 @@ public class SimpleQueryableIndex implements QueryableIndex
this.dataInterval = dataInterval; this.dataInterval = dataInterval;
this.columnNames = columnNames; this.columnNames = columnNames;
this.availableDimensions = dimNames; this.availableDimensions = dimNames;
this.bitmapFactory = bitmapFactory;
this.columns = columns; this.columns = columns;
this.fileMapper = fileMapper; this.fileMapper = fileMapper;
} }
@ -78,6 +82,12 @@ public class SimpleQueryableIndex implements QueryableIndex
return availableDimensions; return availableDimensions;
} }
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
return bitmapFactory;
}
@Override @Override
public Column getColumn(String columnName) public Column getColumn(String columnName)
{ {

View File

@ -19,15 +19,22 @@
package io.druid.segment.column; package io.druid.segment.column;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
/** /**
*/ */
public interface BitmapIndex public interface BitmapIndex
{ {
public int getCardinality(); public int getCardinality();
public String getValue(int index); public String getValue(int index);
public boolean hasNulls(); public boolean hasNulls();
public ImmutableConciseSet getConciseSet(String value);
public ImmutableConciseSet getConciseSet(int idx); public BitmapFactory getBitmapFactory();
public ImmutableBitmap getBitmap(String value);
public ImmutableBitmap getBitmap(int idx);
} }

View File

@ -0,0 +1,109 @@
/*
* 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.segment.data;
import com.google.common.collect.Ordering;
import com.metamx.collections.bitmap.ImmutableBitmap;
import org.roaringbitmap.IntIterator;
import javax.annotation.Nullable;
import java.util.Iterator;
/**
*/
public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<ImmutableBitmap>
{
private static Ordering<ImmutableBitmap> comparator = new Ordering<ImmutableBitmap>()
{
@Override
public int compare(
ImmutableBitmap set, ImmutableBitmap set1
)
{
if (set.size() == 0 && set1.size() == 0) {
return 0;
}
if (set.size() == 0) {
return -1;
}
if (set1.size() == 0) {
return 1;
}
return set.compareTo(set1);
}
}.nullsFirst();
private final ImmutableBitmap immutableBitmap;
public BitmapCompressedIndexedInts(ImmutableBitmap immutableBitmap)
{
this.immutableBitmap = immutableBitmap;
}
@Override
public int compareTo(@Nullable ImmutableBitmap otherBitmap)
{
return comparator.compare(immutableBitmap, otherBitmap);
}
@Override
public int size()
{
return immutableBitmap.size();
}
@Override
public int get(int index)
{
throw new UnsupportedOperationException("This is really slow, so it's just not supported.");
}
public ImmutableBitmap getImmutableBitmap()
{
return immutableBitmap;
}
@Override
public Iterator<Integer> iterator()
{
return new Iterator<Integer>()
{
IntIterator baseIterator = immutableBitmap.iterator();
@Override
public boolean hasNext()
{
return baseIterator.hasNext();
}
@Override
public Integer next()
{
return baseIterator.next();
}
@Override
public void remove()
{
throw new UnsupportedOperationException();
}
};
}
}

View File

@ -17,31 +17,26 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.metadata; package io.druid.segment.data;
import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.fasterxml.jackson.annotation.JsonSubTypes;
import org.skife.jdbi.v2.tweak.ConnectionFactory; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import java.sql.Connection; /**
import java.sql.DriverManager; */
import java.sql.SQLException; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = ConciseBitmapSerdeFactory.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "concise", value = ConciseBitmapSerdeFactory.class),
@JsonSubTypes.Type(name = "roaring", value = RoaringBitmapSerdeFactory.class)
})
public class DerbyConnectionFactory implements ConnectionFactory public interface BitmapSerdeFactory
{ {
final private String dbName; public static BitmapSerdeFactory DEFAULT_BITMAP_SERDE_FACTORY = new ConciseBitmapSerdeFactory();
public DerbyConnectionFactory(String dbName) { public ObjectStrategy<ImmutableBitmap> getObjectStrategy();
this.dbName = dbName;
}
public Connection openConnection() throws SQLException { public BitmapFactory getBitmapFactory();
final String nsURL=String.format("jdbc:derby://localhost:1527/%s;create=true", dbName);
try {
Class.forName("org.apache.derby.jdbc.ClientDriver");
} catch (Exception e) {
throw Throwables.propagate(e);
}
return DriverManager.getConnection(nsURL);
}
} }

View File

@ -0,0 +1,102 @@
/*
* 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.segment.data;
import com.google.common.collect.Ordering;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ConciseBitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.nio.ByteBuffer;
/**
*/
public class ConciseBitmapSerdeFactory implements BitmapSerdeFactory
{
private static final ObjectStrategy<ImmutableBitmap> objectStrategy = new ImmutableConciseSetObjectStrategy();
private static final BitmapFactory bitmapFactory = new ConciseBitmapFactory();
@Override
public ObjectStrategy<ImmutableBitmap> getObjectStrategy()
{
return objectStrategy;
}
@Override
public BitmapFactory getBitmapFactory()
{
return bitmapFactory;
}
private static Ordering<WrappedImmutableConciseBitmap> conciseComparator = new Ordering<WrappedImmutableConciseBitmap>()
{
@Override
public int compare(
WrappedImmutableConciseBitmap conciseSet, WrappedImmutableConciseBitmap conciseSet1
)
{
if (conciseSet.size() == 0 && conciseSet1.size() == 0) {
return 0;
}
if (conciseSet.size() == 0) {
return -1;
}
if (conciseSet1.size() == 0) {
return 1;
}
return conciseSet.compareTo(conciseSet1);
}
}.nullsFirst();
private static class ImmutableConciseSetObjectStrategy
implements ObjectStrategy<ImmutableBitmap>
{
@Override
public Class<ImmutableBitmap> getClazz()
{
return ImmutableBitmap.class;
}
@Override
public WrappedImmutableConciseBitmap fromByteBuffer(ByteBuffer buffer, int numBytes)
{
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new WrappedImmutableConciseBitmap(new ImmutableConciseSet(readOnlyBuffer));
}
@Override
public byte[] toBytes(ImmutableBitmap val)
{
if (val == null || val.size() == 0) {
return new byte[]{};
}
return val.toBytes();
}
@Override
public int compare(ImmutableBitmap o1, ImmutableBitmap o2)
{
return conciseComparator.compare((WrappedImmutableConciseBitmap) o1, (WrappedImmutableConciseBitmap) o2);
}
}
}

View File

@ -20,6 +20,9 @@
package io.druid.segment.data; package io.druid.segment.data;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap;
import com.metamx.common.ISE;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import it.uniroma3.mat.extendedset.intset.IntSet; import it.uniroma3.mat.extendedset.intset.IntSet;
@ -29,28 +32,28 @@ import java.util.Iterator;
/** /**
*/ */
public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<ConciseCompressedIndexedInts> public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<ImmutableBitmap>
{ {
public static ObjectStrategy<ImmutableConciseSet> objectStrategy = public static ObjectStrategy<ImmutableBitmap> objectStrategy =
new ImmutableConciseSetObjectStrategy(); new ImmutableConciseSetObjectStrategy();
private static Ordering<ImmutableConciseSet> comparator = new Ordering<ImmutableConciseSet>() private static Ordering<ImmutableBitmap> comparator = new Ordering<ImmutableBitmap>()
{ {
@Override @Override
public int compare( public int compare(
@Nullable ImmutableConciseSet conciseSet, @Nullable ImmutableConciseSet conciseSet1 ImmutableBitmap set, ImmutableBitmap set1
) )
{ {
if (conciseSet.size() == 0 && conciseSet1.size() == 0) { if (set.size() == 0 && set1.size() == 0) {
return 0; return 0;
} }
if (conciseSet.size() == 0) { if (set.size() == 0) {
return -1; return -1;
} }
if (conciseSet1.size() == 0) { if (set1.size() == 0) {
return 1; return 1;
} }
return conciseSet.compareTo(conciseSet1); return set.compareTo(set1);
} }
}.nullsFirst(); }.nullsFirst();
@ -62,9 +65,14 @@ public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<Con
} }
@Override @Override
public int compareTo(ConciseCompressedIndexedInts conciseCompressedIndexedInts) public int compareTo(@Nullable ImmutableBitmap conciseCompressedIndexedInts)
{ {
return immutableConciseSet.compareTo(conciseCompressedIndexedInts.getImmutableConciseSet()); // TODO
if (!(conciseCompressedIndexedInts instanceof WrappedImmutableConciseBitmap)) {
throw new ISE("WTF bro! No, bad.");
}
return immutableConciseSet.compareTo(((WrappedImmutableConciseBitmap) conciseCompressedIndexedInts).getBitmap());
} }
@Override @Override
@ -112,24 +120,24 @@ public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<Con
} }
private static class ImmutableConciseSetObjectStrategy private static class ImmutableConciseSetObjectStrategy
implements ObjectStrategy<ImmutableConciseSet> implements ObjectStrategy<ImmutableBitmap>
{ {
@Override @Override
public Class<? extends ImmutableConciseSet> getClazz() public Class<? extends ImmutableBitmap> getClazz()
{ {
return ImmutableConciseSet.class; return WrappedImmutableConciseBitmap.class;
} }
@Override @Override
public ImmutableConciseSet fromByteBuffer(ByteBuffer buffer, int numBytes) public WrappedImmutableConciseBitmap fromByteBuffer(ByteBuffer buffer, int numBytes)
{ {
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new ImmutableConciseSet(readOnlyBuffer); return new WrappedImmutableConciseBitmap(new ImmutableConciseSet(readOnlyBuffer));
} }
@Override @Override
public byte[] toBytes(ImmutableConciseSet val) public byte[] toBytes(ImmutableBitmap val)
{ {
if (val == null || val.size() == 0) { if (val == null || val.size() == 0) {
return new byte[]{}; return new byte[]{};
@ -138,7 +146,7 @@ public class ConciseCompressedIndexedInts implements IndexedInts, Comparable<Con
} }
@Override @Override
public int compare(ImmutableConciseSet o1, ImmutableConciseSet o2) public int compare(ImmutableBitmap o1, ImmutableBitmap o2)
{ {
return comparator.compare(o1, o2); return comparator.compare(o1, o2);
} }

View File

@ -20,6 +20,7 @@
package io.druid.segment.data; package io.druid.segment.data;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -28,9 +29,6 @@ import java.nio.ByteBuffer;
*/ */
public class IndexedRTree implements Comparable<IndexedRTree> public class IndexedRTree implements Comparable<IndexedRTree>
{ {
public static ObjectStrategy<ImmutableRTree> objectStrategy =
new ImmutableRTreeObjectStrategy();
private static Ordering<ImmutableRTree> comparator = new Ordering<ImmutableRTree>() private static Ordering<ImmutableRTree> comparator = new Ordering<ImmutableRTree>()
{ {
@Override @Override
@ -69,9 +67,16 @@ public class IndexedRTree implements Comparable<IndexedRTree>
return immutableRTree; return immutableRTree;
} }
private static class ImmutableRTreeObjectStrategy public static class ImmutableRTreeObjectStrategy
implements ObjectStrategy<ImmutableRTree> implements ObjectStrategy<ImmutableRTree>
{ {
private final BitmapFactory bitmapFactory;
public ImmutableRTreeObjectStrategy(BitmapFactory bitmapFactory)
{
this.bitmapFactory = bitmapFactory;
}
@Override @Override
public Class<? extends ImmutableRTree> getClazz() public Class<? extends ImmutableRTree> getClazz()
{ {
@ -81,10 +86,9 @@ public class IndexedRTree implements Comparable<IndexedRTree>
@Override @Override
public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes) public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes)
{ {
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new ImmutableRTree(readOnlyBuffer); return new ImmutableRTree(readOnlyBuffer, bitmapFactory);
} }
@Override @Override

View File

@ -0,0 +1,103 @@
/*
* 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.segment.data;
import com.google.common.collect.Ordering;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.RoaringBitmapFactory;
import com.metamx.collections.bitmap.WrappedImmutableRoaringBitmap;
import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
import java.nio.ByteBuffer;
/**
*/
public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory
{
private static final ObjectStrategy<ImmutableBitmap> objectStrategy = new ImmutableRoaringBitmapObjectStrategy();
private static final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
@Override
public ObjectStrategy<ImmutableBitmap> getObjectStrategy()
{
return objectStrategy;
}
@Override
public BitmapFactory getBitmapFactory()
{
return bitmapFactory;
}
private static Ordering<WrappedImmutableRoaringBitmap> roaringComparator = new Ordering<WrappedImmutableRoaringBitmap>()
{
@Override
public int compare(
WrappedImmutableRoaringBitmap set1, WrappedImmutableRoaringBitmap set2
)
{
if (set1.size() == 0 && set2.size() == 0) {
return 0;
}
if (set1.size() == 0) {
return -1;
}
if (set2.size() == 0) {
return 1;
}
return set1.compareTo(set2);
}
}.nullsFirst();
private static class ImmutableRoaringBitmapObjectStrategy
implements ObjectStrategy<ImmutableBitmap>
{
@Override
public Class<ImmutableBitmap> getClazz()
{
return ImmutableBitmap.class;
}
@Override
public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes)
{
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
return new WrappedImmutableRoaringBitmap(new ImmutableRoaringBitmap(readOnlyBuffer));
}
@Override
public byte[] toBytes(ImmutableBitmap val)
{
if (val == null || val.size() == 0) {
return new byte[]{};
}
return val.toBytes();
}
@Override
public int compare(ImmutableBitmap o1, ImmutableBitmap o2)
{
return roaringComparator.compare((WrappedImmutableRoaringBitmap) o1, (WrappedImmutableRoaringBitmap) o2);
}
}
}

View File

@ -20,11 +20,11 @@
package io.druid.segment.filter; package io.druid.segment.filter;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.List; import java.util.List;
@ -42,18 +42,18 @@ public class AndFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
{ {
if (filters.size() == 1) { if (filters.size() == 1) {
return filters.get(0).goConcise(selector); return filters.get(0).getBitmapIndex(selector);
} }
List<ImmutableConciseSet> conciseSets = Lists.newArrayList(); List<ImmutableBitmap> bitmaps = Lists.newArrayList();
for (int i = 0; i < filters.size(); i++) { for (int i = 0; i < filters.size(); i++) {
conciseSets.add(filters.get(i).goConcise(selector)); bitmaps.add(filters.get(i).getBitmapIndex(selector));
} }
return ImmutableConciseSet.intersection(conciseSets); return selector.getBitmapFactory().intersection(bitmaps);
} }
@Override @Override

View File

@ -21,13 +21,13 @@ package io.druid.segment.filter;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterable;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -48,23 +48,23 @@ class DimensionPredicateFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
{ {
Indexed<String> dimValues = selector.getDimensionValues(dimension); Indexed<String> dimValues = selector.getDimensionValues(dimension);
if (dimValues == null || dimValues.size() == 0 || predicate == null) { if (dimValues == null || dimValues.size() == 0 || predicate == null) {
return new ImmutableConciseSet(); return selector.getBitmapFactory().makeEmptyImmutableBitmap();
} }
return ImmutableConciseSet.union( return selector.getBitmapFactory().union(
FunctionalIterable.create(dimValues) FunctionalIterable.create(dimValues)
.filter(predicate) .filter(predicate)
.transform( .transform(
new Function<String, ImmutableConciseSet>() new Function<String, ImmutableBitmap>()
{ {
@Override @Override
public ImmutableConciseSet apply(@Nullable String input) public ImmutableBitmap apply(@Nullable String input)
{ {
return selector.getConciseInvertedIndex(dimension, input); return selector.getBitmapIndex(dimension, input);
} }
} }
) )

View File

@ -20,13 +20,13 @@
package io.druid.segment.filter; package io.druid.segment.filter;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.DimExtractionFn;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.List; import java.util.List;
@ -34,8 +34,6 @@ import java.util.List;
*/ */
public class ExtractionFilter implements Filter public class ExtractionFilter implements Filter
{ {
private static final int MAX_SIZE = 50000;
private final String dimension; private final String dimension;
private final String value; private final String value;
private final DimExtractionFn fn; private final DimExtractionFn fn;
@ -67,9 +65,9 @@ public class ExtractionFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
{ {
return new OrFilter(makeFilters(selector)).goConcise(selector); return new OrFilter(makeFilters(selector)).getBitmapIndex(selector);
} }
@Override @Override

View File

@ -21,13 +21,13 @@ package io.druid.segment.filter;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterable;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.data.Indexed; import io.druid.segment.data.Indexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import org.mozilla.javascript.Context; import org.mozilla.javascript.Context;
import org.mozilla.javascript.Function; import org.mozilla.javascript.Function;
import org.mozilla.javascript.ScriptableObject; import org.mozilla.javascript.ScriptableObject;
@ -46,39 +46,42 @@ public class JavaScriptFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
{ {
final Context cx = Context.enter(); final Context cx = Context.enter();
try { try {
final Indexed<String> dimValues = selector.getDimensionValues(dimension); final Indexed<String> dimValues = selector.getDimensionValues(dimension);
ImmutableConciseSet conciseSet; ImmutableBitmap bitmap;
if (dimValues == null) { if (dimValues == null) {
conciseSet = new ImmutableConciseSet(); bitmap = selector.getBitmapFactory().makeEmptyImmutableBitmap();
} else { } else {
conciseSet = ImmutableConciseSet.union( bitmap = selector.getBitmapFactory().union(
FunctionalIterable.create(dimValues) FunctionalIterable.create(dimValues)
.filter(new Predicate<String>() .filter(
new Predicate<String>()
{ {
@Override @Override
public boolean apply(@Nullable String input) public boolean apply(@Nullable String input)
{ {
return predicate.applyInContext(cx, input); return predicate.applyInContext(cx, input);
} }
}) }
)
.transform( .transform(
new com.google.common.base.Function<String, ImmutableConciseSet>() new com.google.common.base.Function<String, ImmutableBitmap>()
{ {
@Override @Override
public ImmutableConciseSet apply(@Nullable String input) public ImmutableBitmap apply(@Nullable String input)
{ {
return selector.getConciseInvertedIndex(dimension, input); return selector.getBitmapIndex(dimension, input);
} }
} }
) )
); );
} }
return conciseSet; return bitmap;
} finally { }
finally {
Context.exit(); Context.exit();
} }
} }
@ -107,7 +110,8 @@ public class JavaScriptFilter implements Filter
scope = cx.initStandardObjects(); scope = cx.initStandardObjects();
fnApply = cx.compileFunction(scope, script, "script", 1, null); fnApply = cx.compileFunction(scope, script, "script", 1, null);
} finally { }
finally {
Context.exit(); Context.exit();
} }
} }
@ -119,7 +123,8 @@ public class JavaScriptFilter implements Filter
final Context cx = Context.enter(); final Context cx = Context.enter();
try { try {
return applyInContext(cx, input); return applyInContext(cx, input);
} finally { }
finally {
Context.exit(); Context.exit();
} }

View File

@ -19,11 +19,11 @@
package io.druid.segment.filter; package io.druid.segment.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
/** /**
*/ */
@ -39,10 +39,10 @@ public class NotFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
{ {
return ImmutableConciseSet.complement( return selector.getBitmapFactory().complement(
baseFilter.goConcise(selector), baseFilter.getBitmapIndex(selector),
selector.getNumRows() selector.getNumRows()
); );
} }

View File

@ -20,11 +20,11 @@
package io.druid.segment.filter; package io.druid.segment.filter;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.List; import java.util.List;
@ -46,18 +46,18 @@ public class OrFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
{ {
if (filters.size() == 1) { if (filters.size() == 1) {
return filters.get(0).goConcise(selector); return filters.get(0).getBitmapIndex(selector);
} }
List<ImmutableConciseSet> conciseSets = Lists.newArrayList(); List<ImmutableBitmap> bitmaps = Lists.newArrayList();
for (int i = 0; i < filters.size(); i++) { for (int i = 0; i < filters.size(); i++) {
conciseSets.add(filters.get(i).goConcise(selector)); bitmaps.add(filters.get(i).getBitmapIndex(selector));
} }
return ImmutableConciseSet.union(conciseSets); return selector.getBitmapFactory().union(bitmaps);
} }
@Override @Override

View File

@ -19,11 +19,11 @@
package io.druid.segment.filter; package io.druid.segment.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
/** /**
*/ */
@ -42,9 +42,9 @@ public class SelectorFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
{ {
return selector.getConciseInvertedIndex(dimension, value); return selector.getBitmapIndex(dimension, value);
} }
@Override @Override

View File

@ -18,12 +18,12 @@
*/ */
package io.druid.segment.filter; package io.druid.segment.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.search.Bound; import com.metamx.collections.spatial.search.Bound;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
/** /**
*/ */
@ -42,9 +42,13 @@ public class SpatialFilter implements Filter
} }
@Override @Override
public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
{ {
return ImmutableConciseSet.union(selector.getSpatialIndex(dimension).search(bound)); Iterable<ImmutableBitmap> search = selector.getSpatialIndex(dimension).search(bound);
for (ImmutableBitmap immutableBitmap : search) {
System.out.println(immutableBitmap);
}
return selector.getBitmapFactory().union(search);
} }
@Override @Override

View File

@ -21,6 +21,8 @@ package io.druid.segment.incremental;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.MutableBitmap;
import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.segment.IndexableAdapter; import io.druid.segment.IndexableAdapter;
@ -31,9 +33,8 @@ import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.IndexedIterable;
import io.druid.segment.data.ListIndexed; import io.druid.segment.data.ListIndexed;
import it.uniroma3.mat.extendedset.intset.ConciseSet;
import it.uniroma3.mat.extendedset.intset.IntSet;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.roaringbitmap.IntIterator;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.Iterator; import java.util.Iterator;
@ -46,10 +47,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter
private static final Logger log = new Logger(IncrementalIndexAdapter.class); private static final Logger log = new Logger(IncrementalIndexAdapter.class);
private final Interval dataInterval; private final Interval dataInterval;
private final IncrementalIndex index; private final IncrementalIndex index;
private final Map<String, Map<String, ConciseSet>> invertedIndexes; private final Map<String, Map<String, MutableBitmap>> invertedIndexes;
public IncrementalIndexAdapter( public IncrementalIndexAdapter(
Interval dataInterval, IncrementalIndex index Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory
) )
{ {
this.dataInterval = dataInterval; this.dataInterval = dataInterval;
@ -58,7 +59,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
this.invertedIndexes = Maps.newHashMap(); this.invertedIndexes = Maps.newHashMap();
for (String dimension : index.getDimensions()) { for (String dimension : index.getDimensions()) {
invertedIndexes.put(dimension, Maps.<String, ConciseSet>newHashMap()); invertedIndexes.put(dimension, Maps.<String, MutableBitmap>newHashMap());
} }
int rowNum = 0; int rowNum = 0;
@ -67,10 +68,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter
for (String dimension : index.getDimensions()) { for (String dimension : index.getDimensions()) {
int dimIndex = index.getDimensionIndex(dimension); int dimIndex = index.getDimensionIndex(dimension);
Map<String, ConciseSet> conciseSets = invertedIndexes.get(dimension); Map<String, MutableBitmap> bitmapIndexes = invertedIndexes.get(dimension);
if (conciseSets == null || dims == null) { if (bitmapIndexes == null || dims == null) {
log.error("conciseSets and dims are null!"); log.error("bitmapIndexes and dims are null!");
continue; continue;
} }
if (dimIndex >= dims.length || dims[dimIndex] == null) { if (dimIndex >= dims.length || dims[dimIndex] == null) {
@ -78,15 +79,15 @@ public class IncrementalIndexAdapter implements IndexableAdapter
} }
for (String dimValue : dims[dimIndex]) { for (String dimValue : dims[dimIndex]) {
ConciseSet conciseSet = conciseSets.get(dimValue); MutableBitmap mutableBitmap = bitmapIndexes.get(dimValue);
if (conciseSet == null) { if (mutableBitmap == null) {
conciseSet = new ConciseSet(); mutableBitmap = bitmapFactory.makeEmptyMutableBitmap();
conciseSets.put(dimValue, conciseSet); bitmapIndexes.put(dimValue, mutableBitmap);
} }
try { try {
conciseSet.add(rowNum); mutableBitmap.add(rowNum);
} }
catch (Exception e) { catch (Exception e) {
log.info(e.toString()); log.info(e.toString());
@ -220,17 +221,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter
} }
@Override @Override
public IndexedInts getInverteds(String dimension, String value) public IndexedInts getBitmapIndex(String dimension, String value)
{ {
Map<String, ConciseSet> dimInverted = invertedIndexes.get(dimension); Map<String, MutableBitmap> dimInverted = invertedIndexes.get(dimension);
if (dimInverted == null) { if (dimInverted == null) {
return new EmptyIndexedInts(); return new EmptyIndexedInts();
} }
final ConciseSet conciseSet = dimInverted.get(value); final MutableBitmap bitmapIndex = dimInverted.get(value);
if (conciseSet == null) { if (bitmapIndex == null) {
return new EmptyIndexedInts(); return new EmptyIndexedInts();
} }
@ -239,7 +240,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
@Override @Override
public int size() public int size()
{ {
return conciseSet.size(); return bitmapIndex.size();
} }
@Override @Override
@ -253,7 +254,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
{ {
return new Iterator<Integer>() return new Iterator<Integer>()
{ {
IntSet.IntIterator baseIter = conciseSet.iterator(); IntIterator baseIter = bitmapIndex.iterator();
@Override @Override
public boolean hasNext() public boolean hasNext()

View File

@ -20,23 +20,26 @@
package io.druid.segment.serde; package io.druid.segment.serde;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.BitmapIndex;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
/** /**
*/ */
public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex> public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex>
{ {
private static final ImmutableConciseSet EMPTY_SET = new ImmutableConciseSet(); private final BitmapFactory bitmapFactory;
private final GenericIndexed<ImmutableBitmap> bitmaps;
private final GenericIndexed<ImmutableConciseSet> bitmaps;
private final GenericIndexed<String> dictionary; private final GenericIndexed<String> dictionary;
public BitmapIndexColumnPartSupplier( public BitmapIndexColumnPartSupplier(
GenericIndexed<ImmutableConciseSet> bitmaps, BitmapFactory bitmapFactory,
GenericIndexed<ImmutableBitmap> bitmaps,
GenericIndexed<String> dictionary GenericIndexed<String> dictionary
) { )
{
this.bitmapFactory = bitmapFactory;
this.bitmaps = bitmaps; this.bitmaps = bitmaps;
this.dictionary = dictionary; this.dictionary = dictionary;
} }
@ -65,22 +68,28 @@ public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex>
} }
@Override @Override
public ImmutableConciseSet getConciseSet(String value) public BitmapFactory getBitmapFactory()
{ {
final int index = dictionary.indexOf(value); return bitmapFactory;
return getConciseSet(index);
} }
@Override @Override
public ImmutableConciseSet getConciseSet(int idx) public ImmutableBitmap getBitmap(String value)
{ {
if (idx < 0) { final int index = dictionary.indexOf(value);
return EMPTY_SET;
return getBitmap(index);
} }
final ImmutableConciseSet bitmap = bitmaps.get(idx); @Override
return bitmap == null ? EMPTY_SET : bitmap; public ImmutableBitmap getBitmap(int idx)
{
if (idx < 0) {
return bitmapFactory.makeEmptyImmutableBitmap();
}
final ImmutableBitmap bitmap = bitmaps.get(idx);
return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap;
} }
}; };
} }

View File

@ -22,18 +22,18 @@ package io.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.collections.spatial.ImmutableRTree;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType; import io.druid.segment.column.ValueType;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.ByteBufferSerializer; import io.druid.segment.data.ByteBufferSerializer;
import io.druid.segment.data.ConciseCompressedIndexedInts;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.IndexedRTree;
import io.druid.segment.data.VSizeIndexed; import io.druid.segment.data.VSizeIndexed;
import io.druid.segment.data.VSizeIndexedInts; import io.druid.segment.data.VSizeIndexedInts;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -44,11 +44,12 @@ import java.nio.channels.WritableByteChannel;
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
{ {
private final boolean isSingleValued; private final boolean isSingleValued;
private final BitmapSerdeFactory bitmapSerdeFactory;
private final GenericIndexed<String> dictionary; private final GenericIndexed<String> dictionary;
private final VSizeIndexedInts singleValuedColumn; private final VSizeIndexedInts singleValuedColumn;
private final VSizeIndexed multiValuedColumn; private final VSizeIndexed multiValuedColumn;
private final GenericIndexed<ImmutableConciseSet> bitmaps; private final GenericIndexed<ImmutableBitmap> bitmaps;
private final ImmutableRTree spatialIndex; private final ImmutableRTree spatialIndex;
private final long size; private final long size;
@ -57,11 +58,14 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
GenericIndexed<String> dictionary, GenericIndexed<String> dictionary,
VSizeIndexedInts singleValCol, VSizeIndexedInts singleValCol,
VSizeIndexed multiValCol, VSizeIndexed multiValCol,
GenericIndexed<ImmutableConciseSet> bitmaps, BitmapSerdeFactory bitmapSerdeFactory,
GenericIndexed<ImmutableBitmap> bitmaps,
ImmutableRTree spatialIndex ImmutableRTree spatialIndex
) )
{ {
this.isSingleValued = multiValCol == null; this.isSingleValued = multiValCol == null;
this.bitmapSerdeFactory = bitmapSerdeFactory;
this.dictionary = dictionary; this.dictionary = dictionary;
this.singleValuedColumn = singleValCol; this.singleValuedColumn = singleValCol;
this.multiValuedColumn = multiValCol; this.multiValuedColumn = multiValCol;
@ -86,10 +90,14 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
@JsonCreator @JsonCreator
public DictionaryEncodedColumnPartSerde( public DictionaryEncodedColumnPartSerde(
@JsonProperty("isSingleValued") boolean isSingleValued @JsonProperty("isSingleValued") boolean isSingleValued,
@JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory
) )
{ {
this.isSingleValued = isSingleValued; this.isSingleValued = isSingleValued;
this.bitmapSerdeFactory = bitmapSerdeFactory == null
? BitmapSerdeFactory.DEFAULT_BITMAP_SERDE_FACTORY
: bitmapSerdeFactory;
this.dictionary = null; this.dictionary = null;
this.singleValuedColumn = null; this.singleValuedColumn = null;
@ -105,6 +113,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
return isSingleValued; return isSingleValued;
} }
@JsonProperty
public BitmapSerdeFactory getBitmapSerdeFactory()
{
return bitmapSerdeFactory;
}
@Override @Override
public long numBytes() public long numBytes()
{ {
@ -135,7 +149,11 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
} }
if (spatialIndex != null) { if (spatialIndex != null) {
ByteBufferSerializer.writeToChannel(spatialIndex, IndexedRTree.objectStrategy, channel); ByteBufferSerializer.writeToChannel(
spatialIndex,
new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()),
channel
);
} }
} }
@ -153,23 +171,43 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer); singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer);
multiValuedColumn = null; multiValuedColumn = null;
builder.setHasMultipleValues(false) builder.setHasMultipleValues(false)
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null, columnConfig.columnCacheSizeBytes())); .setDictionaryEncodedColumn(
new DictionaryEncodedColumnSupplier(
dictionary,
singleValuedColumn,
null,
columnConfig.columnCacheSizeBytes()
)
);
} else { } else {
singleValuedColumn = null; singleValuedColumn = null;
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer); multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
builder.setHasMultipleValues(true) builder.setHasMultipleValues(true)
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn, columnConfig.columnCacheSizeBytes())); .setDictionaryEncodedColumn(
new DictionaryEncodedColumnSupplier(
dictionary,
null,
multiValuedColumn,
columnConfig.columnCacheSizeBytes()
)
);
} }
GenericIndexed<ImmutableConciseSet> bitmaps = GenericIndexed.read( GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(
buffer, ConciseCompressedIndexedInts.objectStrategy buffer, bitmapSerdeFactory.getObjectStrategy()
);
builder.setBitmapIndex(
new BitmapIndexColumnPartSupplier(
bitmapSerdeFactory.getBitmapFactory(),
bitmaps,
dictionary
)
); );
builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary));
ImmutableRTree spatialIndex = null; ImmutableRTree spatialIndex = null;
if (buffer.hasRemaining()) { if (buffer.hasRemaining()) {
spatialIndex = ByteBufferSerializer.read( spatialIndex = ByteBufferSerializer.read(
buffer, IndexedRTree.objectStrategy buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory())
); );
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(spatialIndex)); builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(spatialIndex));
} }
@ -178,6 +216,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
dictionary, dictionary,
singleValuedColumn, singleValuedColumn,
multiValuedColumn, multiValuedColumn,
bitmapSerdeFactory,
bitmaps, bitmaps,
spatialIndex spatialIndex
); );

View File

@ -19,6 +19,8 @@
package io.druid.segment; package io.druid.segment;
import com.metamx.collections.bitmap.ConciseBitmapFactory;
import com.metamx.collections.bitmap.WrappedImmutableConciseBitmap;
import io.druid.segment.data.Offset; import io.druid.segment.data.Offset;
import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ConciseSet;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
@ -27,7 +29,7 @@ import org.junit.Test;
/** /**
*/ */
public class ConciseOffsetTest public class BitmapOffsetTest
{ {
@Test @Test
public void testSanity() throws Exception public void testSanity() throws Exception
@ -40,7 +42,7 @@ public class ConciseOffsetTest
ImmutableConciseSet set = ImmutableConciseSet.newImmutableFromMutable(mutableSet); ImmutableConciseSet set = ImmutableConciseSet.newImmutableFromMutable(mutableSet);
ConciseOffset offset = new ConciseOffset(set); BitmapOffset offset = new BitmapOffset(new ConciseBitmapFactory(), new WrappedImmutableConciseBitmap(set));
int count = 0; int count = 0;
while (offset.withinBounds()) { while (offset.withinBounds()) {

View File

@ -21,6 +21,7 @@ package io.druid.segment;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ConciseBitmapFactory;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.query.TestQueryRunners; import io.druid.query.TestQueryRunners;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
@ -47,9 +48,18 @@ public class EmptyIndexTest
} }
tmpDir.deleteOnExit(); tmpDir.deleteOnExit();
IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0], TestQueryRunners.pool); IncrementalIndex emptyIndex = new IncrementalIndex(
IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); 0,
IndexMaker.merge( QueryGranularity.NONE,
new AggregatorFactory[0],
TestQueryRunners.pool
);
IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(
new Interval("2012-08-01/P3D"),
emptyIndex,
new ConciseBitmapFactory()
);
IndexMerger.merge(
Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter), Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter),
new AggregatorFactory[0], new AggregatorFactory[0],
tmpDir tmpDir

View File

@ -39,8 +39,12 @@ import java.util.Arrays;
/** /**
*/ */
public class IndexMakerTest public class IndexMergerTest
{ {
static {
}
@Test @Test
public void testPersistCaseInsensitive() throws Exception public void testPersistCaseInsensitive() throws Exception
{ {
@ -50,7 +54,7 @@ public class IndexMakerTest
final File tempDir = Files.createTempDir(); final File tempDir = Files.createTempDir();
try { try {
QueryableIndex index = IndexIO.loadIndex(IndexMaker.persist(toPersist, tempDir)); QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir));
Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions()));
@ -89,20 +93,20 @@ public class IndexMakerTest
final File tempDir2 = Files.createTempDir(); final File tempDir2 = Files.createTempDir();
final File mergedDir = Files.createTempDir(); final File mergedDir = Files.createTempDir();
try { try {
QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tempDir1)); QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1));
Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions()));
Assert.assertEquals(2, index1.getColumnNames().size()); Assert.assertEquals(2, index1.getColumnNames().size());
QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist2, tempDir2)); QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2));
Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions()));
Assert.assertEquals(2, index2.getColumnNames().size()); Assert.assertEquals(2, index2.getColumnNames().size());
QueryableIndex merged = IndexIO.loadIndex( QueryableIndex merged = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Arrays.asList(index1, index2), Arrays.asList(index1, index2),
new AggregatorFactory[]{}, new AggregatorFactory[]{},
mergedDir mergedDir
@ -146,10 +150,10 @@ public class IndexMakerTest
) )
); );
final QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir1)); final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1));
final QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir2)); final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2));
final QueryableIndex merged = IndexIO.loadIndex( final QueryableIndex merged = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3)
); );
Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());

View File

@ -179,11 +179,11 @@ public class SchemalessIndex
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMaker.persist(top, topFile); IndexMerger.persist(top, topFile);
IndexMaker.persist(bottom, bottomFile); IndexMerger.persist(bottom, bottomFile);
mergedIndex = io.druid.segment.IndexIO.loadIndex( mergedIndex = io.druid.segment.IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile
) )
); );
@ -225,7 +225,7 @@ public class SchemalessIndex
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
QueryableIndex index = IndexIO.loadIndex( QueryableIndex index = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile
) )
); );
@ -262,7 +262,7 @@ public class SchemalessIndex
} }
QueryableIndex index = IndexIO.loadIndex( QueryableIndex index = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile)
); );
return index; return index;
@ -343,7 +343,7 @@ public class SchemalessIndex
tmpFile.mkdirs(); tmpFile.mkdirs();
tmpFile.deleteOnExit(); tmpFile.deleteOnExit();
IndexMaker.persist(rowIndex, tmpFile); IndexMerger.persist(rowIndex, tmpFile);
rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile)); rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile));
} }
} }
@ -403,7 +403,7 @@ public class SchemalessIndex
theFile.mkdirs(); theFile.mkdirs();
theFile.deleteOnExit(); theFile.deleteOnExit();
filesToMap.add(theFile); filesToMap.add(theFile);
IndexMaker.persist(index, theFile); IndexMerger.persist(index, theFile);
} }
return filesToMap; return filesToMap;
@ -463,7 +463,7 @@ public class SchemalessIndex
); );
} }
return IndexIO.loadIndex(IndexMaker.append(adapters, mergedFile)); return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile));
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -482,7 +482,7 @@ public class SchemalessIndex
List<File> filesToMap = makeFilesToMap(tmpFile, files); List<File> filesToMap = makeFilesToMap(tmpFile, files);
return IndexIO.loadIndex( return IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Lists.newArrayList( Lists.newArrayList(
Iterables.transform( Iterables.transform(
filesToMap, filesToMap,

View File

@ -1169,6 +1169,7 @@ public class SchemalessTestFull
) )
); );
/* Uncomment when Druid support for nulls/empty strings is actually consistent
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList( List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"), new DateTime("2011-01-12T00:00:00.000Z"),
@ -1205,6 +1206,43 @@ public class SchemalessTestFull
) )
) )
); );
*/
List<Result<TopNResultValue>> expectedTopNResults = Arrays.asList(
new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new TopNResultValue(
Arrays.<Map<String, Object>>asList(
ImmutableMap.<String, Object>builder()
.put("market", "spot")
.put("rows", 4L)
.put("index", 400.0D)
.put("addRowsIndexConstant", 405.0D)
.put("uniques", 0.0D)
.put("maxIndex", 100.0)
.put("minIndex", 100.0)
.build(),
ImmutableMap.<String, Object>builder()
.put("market", "")
.put("rows", 3L)
.put("index", 200.0D)
.put("addRowsIndexConstant", 204.0D)
.put("uniques", 0.0)
.put("maxIndex", 100.0)
.put("minIndex", 0.0)
.build(),
ImmutableMap.<String, Object>builder()
.put("market", "total_market")
.put("rows", 2L)
.put("index", 200.0D)
.put("addRowsIndexConstant", 203.0D)
.put("uniques", UNIQUES_1)
.put("maxIndex", 100.0)
.put("minIndex", 100.0)
.build()
)
)
)
);
List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList( List<Result<TopNResultValue>> expectedFilteredTopNResults = Arrays.asList(
new Result<TopNResultValue>( new Result<TopNResultValue>(

View File

@ -132,11 +132,11 @@ public class TestIndex
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMaker.persist(top, DATA_INTERVAL, topFile); IndexMerger.persist(top, DATA_INTERVAL, topFile);
IndexMaker.persist(bottom, DATA_INTERVAL, bottomFile); IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile);
mergedRealtime = IndexIO.loadIndex( mergedRealtime = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)),
METRIC_AGGS, METRIC_AGGS,
mergedFile mergedFile
@ -243,7 +243,7 @@ public class TestIndex
someTmpFile.mkdirs(); someTmpFile.mkdirs();
someTmpFile.deleteOnExit(); someTmpFile.deleteOnExit();
IndexMaker.persist(index, someTmpFile); IndexMerger.persist(index, someTmpFile);
return IndexIO.loadIndex(someTmpFile); return IndexIO.loadIndex(someTmpFile);
} }
catch (IOException e) { catch (IOException e) {

View File

@ -46,7 +46,7 @@ import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.IncrementalIndexSegment;
import io.druid.segment.IndexIO; import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker; import io.druid.segment.IndexMerger;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment; import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.Segment; import io.druid.segment.Segment;
@ -232,7 +232,7 @@ public class SpatialFilterBonusTest
tmpFile.mkdirs(); tmpFile.mkdirs();
tmpFile.deleteOnExit(); tmpFile.deleteOnExit();
IndexMaker.persist(theIndex, tmpFile); IndexMerger.persist(theIndex, tmpFile);
return IndexIO.loadIndex(tmpFile); return IndexIO.loadIndex(tmpFile);
} }
@ -412,12 +412,12 @@ public class SpatialFilterBonusTest
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMaker.persist(first, DATA_INTERVAL, firstFile); IndexMerger.persist(first, DATA_INTERVAL, firstFile);
IndexMaker.persist(second, DATA_INTERVAL, secondFile); IndexMerger.persist(second, DATA_INTERVAL, secondFile);
IndexMaker.persist(third, DATA_INTERVAL, thirdFile); IndexMerger.persist(third, DATA_INTERVAL, thirdFile);
QueryableIndex mergedRealtime = IndexIO.loadIndex( QueryableIndex mergedRealtime = IndexIO.loadIndex(
IndexMaker.mergeQueryableIndex( IndexMerger.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)),
METRIC_AGGS, METRIC_AGGS,
mergedFile mergedFile

View File

@ -21,12 +21,12 @@ package io.druid.guice;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Module; import com.google.inject.Module;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataRuleManagerConfig; import io.druid.metadata.MetadataRuleManagerConfig;
import io.druid.metadata.MetadataSegmentManagerConfig; import io.druid.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig; import io.druid.metadata.MetadataStorageTablesConfig;
public class MetadataDbConfigModule implements Module public class MetadataConfigModule implements Module
{ {
@Override @Override
public void configure(Binder binder) public void configure(Binder binder)

View File

@ -34,6 +34,8 @@ import io.druid.metadata.MetadataSegmentManagerProvider;
import io.druid.metadata.MetadataSegmentPublisher; import io.druid.metadata.MetadataSegmentPublisher;
import io.druid.metadata.MetadataSegmentPublisherProvider; import io.druid.metadata.MetadataSegmentPublisherProvider;
import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageProvider;
import io.druid.metadata.NoopMetadataStorageProvider;
import io.druid.metadata.SQLMetadataConnector; import io.druid.metadata.SQLMetadataConnector;
import io.druid.metadata.SQLMetadataRuleManager; import io.druid.metadata.SQLMetadataRuleManager;
import io.druid.metadata.SQLMetadataRuleManagerProvider; import io.druid.metadata.SQLMetadataRuleManagerProvider;
@ -62,6 +64,13 @@ public class SQLMetadataStorageDruidModule implements Module
PolyBind.createChoiceWithDefault( PolyBind.createChoiceWithDefault(
binder, PROPERTY, Key.get(MetadataStorageConnector.class), null, defaultPropertyValue binder, PROPERTY, Key.get(MetadataStorageConnector.class), null, defaultPropertyValue
); );
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataStorageProvider.class),
Key.get(NoopMetadataStorageProvider.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault( PolyBind.createChoiceWithDefault(
binder, PROPERTY, Key.get(SQLMetadataConnector.class), null, defaultPropertyValue binder, PROPERTY, Key.get(SQLMetadataConnector.class), null, defaultPropertyValue
); );

View File

@ -36,7 +36,7 @@ import io.druid.curator.CuratorModule;
import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.DiscoveryModule;
import io.druid.guice.AWSModule; import io.druid.guice.AWSModule;
import io.druid.guice.AnnouncerModule; import io.druid.guice.AnnouncerModule;
import io.druid.guice.DerbyMetadataStorageDruidModule; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule;
import io.druid.guice.DruidProcessingModule; import io.druid.guice.DruidProcessingModule;
import io.druid.guice.DruidSecondaryModule; import io.druid.guice.DruidSecondaryModule;
import io.druid.guice.ExtensionsConfig; import io.druid.guice.ExtensionsConfig;
@ -45,7 +45,7 @@ import io.druid.guice.IndexingServiceDiscoveryModule;
import io.druid.guice.JacksonConfigManagerModule; import io.druid.guice.JacksonConfigManagerModule;
import io.druid.guice.LifecycleModule; import io.druid.guice.LifecycleModule;
import io.druid.guice.LocalDataStorageDruidModule; import io.druid.guice.LocalDataStorageDruidModule;
import io.druid.guice.MetadataDbConfigModule; import io.druid.guice.MetadataConfigModule;
import io.druid.guice.ParsersModule; import io.druid.guice.ParsersModule;
import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryRunnerFactoryModule;
import io.druid.guice.QueryableModule; import io.druid.guice.QueryableModule;
@ -333,7 +333,7 @@ public class Initialization
new QueryRunnerFactoryModule(), new QueryRunnerFactoryModule(),
new DiscoveryModule(), new DiscoveryModule(),
new ServerViewModule(), new ServerViewModule(),
new MetadataDbConfigModule(), new MetadataConfigModule(),
new DerbyMetadataStorageDruidModule(), new DerbyMetadataStorageDruidModule(),
new JacksonConfigManagerModule(), new JacksonConfigManagerModule(),
new IndexingServiceDiscoveryModule(), new IndexingServiceDiscoveryModule(),

View File

@ -0,0 +1,20 @@
package io.druid.metadata;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import io.druid.guice.ManageLifecycle;
public abstract class MetadataStorage
{
@LifecycleStart
public void start()
{
// do nothing
}
@LifecycleStop
public void stop()
{
// do nothing
}
}

View File

@ -0,0 +1,26 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2014 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.metadata;
import com.google.inject.Provider;
public interface MetadataStorageProvider extends Provider<MetadataStorage>
{
}

View File

@ -0,0 +1,29 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2014 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.metadata;
public class NoopMetadataStorageProvider implements MetadataStorageProvider
{
@Override
public MetadataStorage get()
{
return new MetadataStorage() {};
}
}

View File

@ -75,7 +75,9 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
*/ */
protected abstract String getSerialType(); protected abstract String getSerialType();
protected abstract boolean tableExists(Handle handle, final String tableName); public String getValidationQuery() { return "SELECT 1"; }
public abstract boolean tableExists(Handle handle, final String tableName);
protected boolean isTransientException(Throwable e) { protected boolean isTransientException(Throwable e) {
return false; return false;
@ -367,7 +369,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
String uri = connectorConfig.getConnectURI(); String uri = connectorConfig.getConnectURI();
dataSource.setUrl(uri); dataSource.setUrl(uri);
dataSource.setValidationQuery(connectorConfig.getValidationQuery()); dataSource.setValidationQuery(getValidationQuery());
dataSource.setTestOnBorrow(true); dataSource.setTestOnBorrow(true);
return dataSource; return dataSource;

View File

@ -17,17 +17,16 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.metadata; package io.druid.metadata.storage.derby;
import com.google.api.client.repackaged.com.google.common.base.Throwables;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.apache.derby.drda.NetworkServerControl; import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.SQLMetadataConnector;
import org.apache.commons.dbcp2.BasicDataSource;
import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.tweak.ConnectionFactory;
import java.net.InetAddress;
public class DerbyConnector extends SQLMetadataConnector public class DerbyConnector extends SQLMetadataConnector
{ {
@ -37,17 +36,27 @@ public class DerbyConnector extends SQLMetadataConnector
@Inject @Inject
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables) public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
{ {
this(config, dbTables, new DBI(getConnectionFactory("druidDerbyDb"))); super(config, dbTables);
final BasicDataSource datasource = getDatasource();
datasource.setDriverClassLoader(getClass().getClassLoader());
datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver");
this.dbi = new DBI(datasource);
} }
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables, DBI dbi) public DerbyConnector(
Supplier<MetadataStorageConnectorConfig> config,
Supplier<MetadataStorageTablesConfig> dbTables,
DBI dbi
)
{ {
super(config, dbTables); super(config, dbTables);
this.dbi = dbi; this.dbi = dbi;
} }
@Override @Override
protected boolean tableExists(Handle handle, String tableName) public boolean tableExists(Handle handle, String tableName)
{ {
return !handle.createQuery("select * from SYS.SYSTABLES where tablename = :tableName") return !handle.createQuery("select * from SYS.SYSTABLES where tablename = :tableName")
.bind("tableName", tableName.toUpperCase()) .bind("tableName", tableName.toUpperCase())
@ -64,14 +73,6 @@ public class DerbyConnector extends SQLMetadataConnector
@Override @Override
public DBI getDBI() { return dbi; } public DBI getDBI() { return dbi; }
private static ConnectionFactory getConnectionFactory(String dbName) @Override
{ public String getValidationQuery() { return "VALUES 1"; }
try {
NetworkServerControl server = new NetworkServerControl(InetAddress.getByName("localhost"),1527);
server.start(null);
} catch (Exception e) {
throw Throwables.propagate(e);
}
return new DerbyConnectionFactory(dbName);
}
} }

View File

@ -0,0 +1,78 @@
/*
* 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.metadata.storage.derby;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import io.druid.guice.ManageLifecycle;
import io.druid.metadata.MetadataStorage;
import io.druid.metadata.MetadataStorageConnectorConfig;
import org.apache.derby.drda.NetworkServerControl;
import java.net.InetAddress;
@ManageLifecycle
public class DerbyMetadataStorage extends MetadataStorage
{
private static final Logger log = new Logger(DerbyMetadataStorage.class);
private final NetworkServerControl server;
@Inject
public DerbyMetadataStorage(MetadataStorageConnectorConfig config)
{
try {
this.server = new NetworkServerControl(InetAddress.getByName(config.getHost()), config.getPort());
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
@LifecycleStart
public void start()
{
try {
log.info("Starting Derby Metadata Storage");
server.start(null);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
@LifecycleStop
public void stop()
{
try {
server.shutdown();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -17,15 +17,16 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.guice; package io.druid.metadata.storage.derby;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Key; import com.google.inject.Key;
import com.google.inject.Provides; import io.druid.guice.LazySingleton;
import io.druid.metadata.DerbyConnector; import io.druid.guice.PolyBind;
import io.druid.guice.SQLMetadataStorageDruidModule;
import io.druid.metadata.MetadataStorageConnector; import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageProvider;
import io.druid.metadata.SQLMetadataConnector; import io.druid.metadata.SQLMetadataConnector;
import org.skife.jdbi.v2.IDBI;
public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule
{ {
@ -42,6 +43,11 @@ public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModu
createBindingChoices(binder, TYPE); createBindingChoices(binder, TYPE);
super.configure(binder); super.configure(binder);
PolyBind.optionBinder(binder, Key.get(MetadataStorageProvider.class))
.addBinding(TYPE)
.to(DerbyMetadataStorageProvider.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class)) PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class))
.addBinding(TYPE) .addBinding(TYPE)
.to(DerbyConnector.class) .to(DerbyConnector.class)

View File

@ -0,0 +1,42 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2014 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.metadata.storage.derby;
import com.google.inject.Inject;
import io.druid.metadata.MetadataStorage;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageProvider;
public class DerbyMetadataStorageProvider implements MetadataStorageProvider
{
private final DerbyMetadataStorage storage;
@Inject
public DerbyMetadataStorageProvider(MetadataStorageConnectorConfig config)
{
this.storage = new DerbyMetadataStorage(config);
}
@Override
public MetadataStorage get()
{
return storage;
}
}

View File

@ -50,6 +50,10 @@ public class ArbitraryGranularitySpec implements GranularitySpec
this.queryGranularity = queryGranularity; this.queryGranularity = queryGranularity;
this.intervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd()); this.intervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd());
if (inputIntervals == null) {
inputIntervals = Lists.newArrayList();
}
// Insert all intervals // Insert all intervals
for (final Interval inputInterval : inputIntervals) { for (final Interval inputInterval : inputIntervals) {
intervals.add(inputInterval); intervals.add(inputInterval);

View File

@ -20,6 +20,7 @@
package io.druid.metadata; package io.druid.metadata;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import io.druid.metadata.storage.derby.DerbyConnector;
import org.junit.Assert; import org.junit.Assert;
import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException; import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;

View File

@ -29,18 +29,20 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.airlift.command.Command; import io.airlift.command.Command;
import io.druid.client.indexing.IndexingServiceClient; import io.druid.client.indexing.IndexingServiceClient;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataRuleManagerConfig;
import io.druid.metadata.MetadataRuleManagerProvider;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataSegmentManagerProvider;
import io.druid.guice.ConfigProvider; import io.druid.guice.ConfigProvider;
import io.druid.guice.Jerseys; import io.druid.guice.Jerseys;
import io.druid.guice.JsonConfigProvider; import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton; import io.druid.guice.LazySingleton;
import io.druid.guice.LifecycleModule; import io.druid.guice.LifecycleModule;
import io.druid.guice.ManageLifecycle; import io.druid.guice.ManageLifecycle;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataRuleManagerConfig;
import io.druid.metadata.MetadataRuleManagerProvider;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataSegmentManagerProvider;
import io.druid.metadata.MetadataStorage;
import io.druid.metadata.MetadataStorageProvider;
import io.druid.server.coordinator.DruidCoordinator; import io.druid.server.coordinator.DruidCoordinator;
import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.coordinator.DruidCoordinatorConfig;
import io.druid.server.coordinator.LoadQueueTaskMaster; import io.druid.server.coordinator.LoadQueueTaskMaster;
@ -49,9 +51,9 @@ import io.druid.server.http.BackwardsCompatibleInfoResource;
import io.druid.server.http.CoordinatorDynamicConfigsResource; import io.druid.server.http.CoordinatorDynamicConfigsResource;
import io.druid.server.http.CoordinatorRedirectInfo; import io.druid.server.http.CoordinatorRedirectInfo;
import io.druid.server.http.CoordinatorResource; import io.druid.server.http.CoordinatorResource;
import io.druid.server.http.MetadataResource;
import io.druid.server.http.DatasourcesResource; import io.druid.server.http.DatasourcesResource;
import io.druid.server.http.InfoResource; import io.druid.server.http.InfoResource;
import io.druid.server.http.MetadataResource;
import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectFilter;
import io.druid.server.http.RedirectInfo; import io.druid.server.http.RedirectInfo;
import io.druid.server.http.RulesResource; import io.druid.server.http.RulesResource;
@ -93,6 +95,10 @@ public class CliCoordinator extends ServerRunnable
ConfigProvider.bind(binder, DruidCoordinatorConfig.class); ConfigProvider.bind(binder, DruidCoordinatorConfig.class);
binder.bind(MetadataStorage.class)
.toProvider(MetadataStorageProvider.class)
.in(ManageLifecycle.class);
JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class);
JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class);
@ -111,6 +117,7 @@ public class CliCoordinator extends ServerRunnable
binder.bind(DruidCoordinator.class); binder.bind(DruidCoordinator.class);
LifecycleModule.register(binder, MetadataStorage.class);
LifecycleModule.register(binder, DruidCoordinator.class); LifecycleModule.register(binder, DruidCoordinator.class);
binder.bind(JettyServerInitializer.class) binder.bind(JettyServerInitializer.class)