mirror of https://github.com/apache/druid.git
Merge branch 'druid-0.7.x' of github.com:metamx/druid into druid-0.7.x
This commit is contained in:
commit
2f498c3422
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 + '\'' +
|
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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>()
|
||||||
|
|
|
@ -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)
|
||||||
|
|
7
pom.xml
7
pom.xml
|
@ -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>
|
||||||
|
|
|
@ -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"
|
||||||
|
|
|
@ -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>
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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"));
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
{
|
{
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
);
|
);
|
||||||
|
|
|
@ -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()) {
|
|
@ -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
|
||||||
|
|
|
@ -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());
|
|
@ -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,
|
||||||
|
|
|
@ -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>(
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
|
@ -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
|
||||||
);
|
);
|
||||||
|
|
|
@ -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(),
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
|
@ -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>
|
||||||
|
{
|
||||||
|
}
|
|
@ -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() {};
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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)
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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)
|
||||||
|
|
Loading…
Reference in New Issue