mirror of https://github.com/apache/druid.git
Merge pull request #1265 from metamx/dimension-compression
Support for Dimension Compression
This commit is contained in:
commit
281adc0457
|
@ -25,9 +25,9 @@ public class StupidResourceHolder<T> implements ResourceHolder<T>
|
||||||
{
|
{
|
||||||
private final T obj;
|
private final T obj;
|
||||||
|
|
||||||
public static <T> StupidResourceHolder create(T obj)
|
public static <T> StupidResourceHolder<T> create(T obj)
|
||||||
{
|
{
|
||||||
return new StupidResourceHolder(obj);
|
return new StupidResourceHolder<>(obj);
|
||||||
}
|
}
|
||||||
|
|
||||||
public StupidResourceHolder(
|
public StupidResourceHolder(
|
||||||
|
|
|
@ -104,10 +104,24 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|
||||||
|
|
||||||
|property|description|default|required?|
|
|property|description|default|required?|
|
||||||
|--------|-----------|-------|---------|
|
|--------|-----------|-------|---------|
|
||||||
|type|The task type, this should always be "index".|None.||yes|
|
|type|The task type, this should always be "index".|None.|yes|
|
||||||
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment. Set this to -1 to use numShards instead for sharding.|5000000|no|
|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment. Set this to -1 to use numShards instead for sharding.|5000000|no|
|
||||||
|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|500000|no|
|
|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|500000|no|
|
||||||
|numShards|Directly specify the number of shards to create. You can skip the intermediate persist step if you specify the number of shards you want and set targetPartitionSize=-1.|null|no|
|
|numShards|Directly specify the number of shards to create. You can skip the intermediate persist step if you specify the number of shards you want and set targetPartitionSize=-1.|null|no|
|
||||||
|
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
|
||||||
|
|
||||||
|
#### IndexSpec
|
||||||
|
|
||||||
|
The indexSpec defines segment storage format options to be used at indexing
|
||||||
|
time, such as bitmap type, and column compression formats.
|
||||||
|
|
||||||
|
The indexSpec is optional and default parameters will be used if not specified.
|
||||||
|
|
||||||
|
|property|description|possible values|default|required?|
|
||||||
|
|--------|-----------|---------------|-------|---------|
|
||||||
|
|bitmap|type of bitmap compression to use for inverted indices.|`"concise"`, `"roaring"`|`"concise"` or the value of `druid.processing.bitmap.type`, if specified|no|
|
||||||
|
|dimensionCompression|compression format for dimension columns (currently only affects single-value dimensions, multi-value dimensions are always uncompressed)|`"uncompressed"`, `"lz4"`, `"lzf"`|`"lz4"`|no|
|
||||||
|
|metricCompression|compression format for metric columns, defaults to LZ4|`"lz4"`, `"lzf"`|`"lz4"`|no|
|
||||||
|
|
||||||
### Index Hadoop Task
|
### Index Hadoop Task
|
||||||
|
|
||||||
|
|
|
@ -17,41 +17,6 @@
|
||||||
|
|
||||||
package io.druid.indexer;
|
package io.druid.indexer;
|
||||||
|
|
||||||
import io.druid.common.utils.JodaUtils;
|
|
||||||
import io.druid.data.input.InputRow;
|
|
||||||
import io.druid.data.input.impl.InputRowParser;
|
|
||||||
import io.druid.granularity.QueryGranularity;
|
|
||||||
import io.druid.guice.GuiceInjectors;
|
|
||||||
import io.druid.guice.JsonConfigProvider;
|
|
||||||
import io.druid.guice.annotations.Self;
|
|
||||||
import io.druid.indexer.partitions.PartitionsSpec;
|
|
||||||
import io.druid.indexer.path.PathSpec;
|
|
||||||
import io.druid.initialization.Initialization;
|
|
||||||
import io.druid.segment.indexing.granularity.GranularitySpec;
|
|
||||||
import io.druid.server.DruidNode;
|
|
||||||
import io.druid.timeline.DataSegment;
|
|
||||||
import io.druid.timeline.partition.ShardSpec;
|
|
||||||
import io.druid.timeline.partition.ShardSpecLookup;
|
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.Charset;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Properties;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.SortedSet;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
import org.joda.time.format.ISODateTimeFormat;
|
|
||||||
|
|
||||||
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.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
@ -71,6 +36,40 @@ import com.google.inject.Key;
|
||||||
import com.google.inject.Module;
|
import com.google.inject.Module;
|
||||||
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.common.utils.JodaUtils;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
|
import io.druid.granularity.QueryGranularity;
|
||||||
|
import io.druid.guice.GuiceInjectors;
|
||||||
|
import io.druid.guice.JsonConfigProvider;
|
||||||
|
import io.druid.guice.annotations.Self;
|
||||||
|
import io.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import io.druid.indexer.path.PathSpec;
|
||||||
|
import io.druid.initialization.Initialization;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
|
import io.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
|
import io.druid.server.DruidNode;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
import io.druid.timeline.partition.ShardSpec;
|
||||||
|
import io.druid.timeline.partition.ShardSpecLookup;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
import org.joda.time.format.ISODateTimeFormat;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.Charset;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.SortedSet;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -247,6 +246,10 @@ public class HadoopDruidIndexerConfig
|
||||||
return schema.getTuningConfig().getPartitionsSpec();
|
return schema.getTuningConfig().getPartitionsSpec();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public IndexSpec getIndexSpec() {
|
||||||
|
return schema.getTuningConfig().getIndexSpec();
|
||||||
|
}
|
||||||
|
|
||||||
public boolean isOverwriteFiles()
|
public boolean isOverwriteFiles()
|
||||||
{
|
{
|
||||||
return schema.getTuningConfig().isOverwriteFiles();
|
return schema.getTuningConfig().isOverwriteFiles();
|
||||||
|
|
|
@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import io.druid.indexer.partitions.HashedPartitionsSpec;
|
import io.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
import io.druid.indexer.partitions.PartitionsSpec;
|
import io.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
|
import io.druid.segment.data.BitmapSerde;
|
||||||
import io.druid.segment.indexing.TuningConfig;
|
import io.druid.segment.indexing.TuningConfig;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
@ -36,6 +38,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
{
|
{
|
||||||
private static final PartitionsSpec DEFAULT_PARTITIONS_SPEC = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec();
|
private static final PartitionsSpec DEFAULT_PARTITIONS_SPEC = HashedPartitionsSpec.makeDefaultHashedPartitionsSpec();
|
||||||
private static final Map<DateTime, List<HadoopyShardSpec>> DEFAULT_SHARD_SPECS = ImmutableMap.<DateTime, List<HadoopyShardSpec>>of();
|
private static final Map<DateTime, List<HadoopyShardSpec>> DEFAULT_SHARD_SPECS = ImmutableMap.<DateTime, List<HadoopyShardSpec>>of();
|
||||||
|
private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec();
|
||||||
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 80000;
|
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 80000;
|
||||||
private static final int DEFAULT_BUFFER_SIZE = 128 * 1024 * 1024;
|
private static final int DEFAULT_BUFFER_SIZE = 128 * 1024 * 1024;
|
||||||
private static final float DEFAULT_AGG_BUFFER_RATIO = 0.5f;
|
private static final float DEFAULT_AGG_BUFFER_RATIO = 0.5f;
|
||||||
|
@ -47,6 +50,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
new DateTime().toString(),
|
new DateTime().toString(),
|
||||||
DEFAULT_PARTITIONS_SPEC,
|
DEFAULT_PARTITIONS_SPEC,
|
||||||
DEFAULT_SHARD_SPECS,
|
DEFAULT_SHARD_SPECS,
|
||||||
|
DEFAULT_INDEX_SPEC,
|
||||||
DEFAULT_ROW_FLUSH_BOUNDARY,
|
DEFAULT_ROW_FLUSH_BOUNDARY,
|
||||||
false,
|
false,
|
||||||
true,
|
true,
|
||||||
|
@ -65,6 +69,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
private final String version;
|
private final String version;
|
||||||
private final PartitionsSpec partitionsSpec;
|
private final PartitionsSpec partitionsSpec;
|
||||||
private final Map<DateTime, List<HadoopyShardSpec>> shardSpecs;
|
private final Map<DateTime, List<HadoopyShardSpec>> shardSpecs;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
private final int rowFlushBoundary;
|
private final int rowFlushBoundary;
|
||||||
private final boolean leaveIntermediate;
|
private final boolean leaveIntermediate;
|
||||||
private final Boolean cleanupOnFailure;
|
private final Boolean cleanupOnFailure;
|
||||||
|
@ -83,6 +88,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
final @JsonProperty("version") String version,
|
final @JsonProperty("version") String version,
|
||||||
final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec,
|
final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec,
|
||||||
final @JsonProperty("shardSpecs") Map<DateTime, List<HadoopyShardSpec>> shardSpecs,
|
final @JsonProperty("shardSpecs") Map<DateTime, List<HadoopyShardSpec>> shardSpecs,
|
||||||
|
final @JsonProperty("indexSpec") IndexSpec indexSpec,
|
||||||
final @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
|
final @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
|
||||||
final @JsonProperty("leaveIntermediate") boolean leaveIntermediate,
|
final @JsonProperty("leaveIntermediate") boolean leaveIntermediate,
|
||||||
final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure,
|
final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure,
|
||||||
|
@ -100,6 +106,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
this.version = version == null ? new DateTime().toString() : version;
|
this.version = version == null ? new DateTime().toString() : version;
|
||||||
this.partitionsSpec = partitionsSpec == null ? DEFAULT_PARTITIONS_SPEC : partitionsSpec;
|
this.partitionsSpec = partitionsSpec == null ? DEFAULT_PARTITIONS_SPEC : partitionsSpec;
|
||||||
this.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs;
|
this.shardSpecs = shardSpecs == null ? DEFAULT_SHARD_SPECS : shardSpecs;
|
||||||
|
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
||||||
this.rowFlushBoundary = maxRowsInMemory == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemory;
|
this.rowFlushBoundary = maxRowsInMemory == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemory;
|
||||||
this.leaveIntermediate = leaveIntermediate;
|
this.leaveIntermediate = leaveIntermediate;
|
||||||
this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure;
|
this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure;
|
||||||
|
@ -139,6 +146,12 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
return shardSpecs;
|
return shardSpecs;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public IndexSpec getIndexSpec()
|
||||||
|
{
|
||||||
|
return indexSpec;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public int getRowFlushBoundary()
|
public int getRowFlushBoundary()
|
||||||
{
|
{
|
||||||
|
@ -210,6 +223,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
version,
|
version,
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
shardSpecs,
|
shardSpecs,
|
||||||
|
indexSpec,
|
||||||
rowFlushBoundary,
|
rowFlushBoundary,
|
||||||
leaveIntermediate,
|
leaveIntermediate,
|
||||||
cleanupOnFailure,
|
cleanupOnFailure,
|
||||||
|
@ -231,6 +245,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
ver,
|
ver,
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
shardSpecs,
|
shardSpecs,
|
||||||
|
indexSpec,
|
||||||
rowFlushBoundary,
|
rowFlushBoundary,
|
||||||
leaveIntermediate,
|
leaveIntermediate,
|
||||||
cleanupOnFailure,
|
cleanupOnFailure,
|
||||||
|
@ -252,6 +267,7 @@ public class HadoopTuningConfig implements TuningConfig
|
||||||
version,
|
version,
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
specs,
|
specs,
|
||||||
|
indexSpec,
|
||||||
rowFlushBoundary,
|
rowFlushBoundary,
|
||||||
leaveIntermediate,
|
leaveIntermediate,
|
||||||
cleanupOnFailure,
|
cleanupOnFailure,
|
||||||
|
|
|
@ -17,61 +17,6 @@
|
||||||
|
|
||||||
package io.druid.indexer;
|
package io.druid.indexer;
|
||||||
|
|
||||||
import com.metamx.common.parsers.ParseException;
|
|
||||||
import io.druid.collections.StupidPool;
|
|
||||||
import io.druid.data.input.InputRow;
|
|
||||||
import io.druid.data.input.Rows;
|
|
||||||
import io.druid.data.input.impl.InputRowParser;
|
|
||||||
import io.druid.offheap.OffheapBufferPool;
|
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
|
||||||
import io.druid.segment.IndexIO;
|
|
||||||
import io.druid.segment.IndexMaker;
|
|
||||||
import io.druid.segment.LoggingProgressIndicator;
|
|
||||||
import io.druid.segment.ProgressIndicator;
|
|
||||||
import io.druid.segment.QueryableIndex;
|
|
||||||
import io.druid.segment.SegmentUtils;
|
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
|
||||||
import io.druid.segment.incremental.OffheapIncrementalIndex;
|
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
|
||||||
import io.druid.timeline.DataSegment;
|
|
||||||
|
|
||||||
import java.io.BufferedOutputStream;
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.FileInputStream;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.net.URI;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.zip.ZipEntry;
|
|
||||||
import java.util.zip.ZipOutputStream;
|
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
|
||||||
import org.apache.hadoop.conf.Configurable;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.BytesWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.io.Writable;
|
|
||||||
import org.apache.hadoop.mapred.InvalidJobConfException;
|
|
||||||
import org.apache.hadoop.mapreduce.Counter;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.Partitioner;
|
|
||||||
import org.apache.hadoop.mapreduce.Reducer;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.CombineTextInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
|
@ -89,6 +34,58 @@ import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.guava.CloseQuietly;
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
import com.metamx.common.parsers.ParseException;
|
||||||
|
import io.druid.collections.StupidPool;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.Rows;
|
||||||
|
import io.druid.data.input.impl.InputRowParser;
|
||||||
|
import io.druid.offheap.OffheapBufferPool;
|
||||||
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.segment.IndexIO;
|
||||||
|
import io.druid.segment.IndexMaker;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
|
import io.druid.segment.LoggingProgressIndicator;
|
||||||
|
import io.druid.segment.ProgressIndicator;
|
||||||
|
import io.druid.segment.QueryableIndex;
|
||||||
|
import io.druid.segment.SegmentUtils;
|
||||||
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
|
import io.druid.segment.incremental.OffheapIncrementalIndex;
|
||||||
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.hadoop.conf.Configurable;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.BytesWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.io.Writable;
|
||||||
|
import org.apache.hadoop.mapred.InvalidJobConfException;
|
||||||
|
import org.apache.hadoop.mapreduce.Counter;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.Partitioner;
|
||||||
|
import org.apache.hadoop.mapreduce.Reducer;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.BufferedOutputStream;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.FileInputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.zip.ZipEntry;
|
||||||
|
import java.util.zip.ZipOutputStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -294,7 +291,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
|
|
||||||
public static class IndexGeneratorReducer extends Reducer<BytesWritable, Writable, BytesWritable, Text>
|
public static class IndexGeneratorReducer extends Reducer<BytesWritable, Writable, BytesWritable, Text>
|
||||||
{
|
{
|
||||||
private HadoopDruidIndexerConfig config;
|
protected HadoopDruidIndexerConfig config;
|
||||||
private List<String> metricNames = Lists.newArrayList();
|
private List<String> metricNames = Lists.newArrayList();
|
||||||
private InputRowParser parser;
|
private InputRowParser parser;
|
||||||
|
|
||||||
|
@ -318,7 +315,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return IndexMaker.persist(
|
return IndexMaker.persist(
|
||||||
index, interval, file, progressIndicator
|
index, interval, file, config.getIndexSpec(), progressIndicator
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -330,7 +327,7 @@ public class IndexGeneratorJob implements Jobby
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return IndexMaker.mergeQueryableIndex(
|
return IndexMaker.mergeQueryableIndex(
|
||||||
indexes, aggs, file, progressIndicator
|
indexes, aggs, file, config.getIndexSpec(), progressIndicator
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,7 @@ package io.druid.indexer;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.segment.BaseProgressIndicator;
|
import io.druid.segment.BaseProgressIndicator;
|
||||||
import io.druid.segment.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.ProgressIndicator;
|
import io.druid.segment.ProgressIndicator;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
|
@ -67,7 +68,7 @@ public class LegacyIndexGeneratorJob extends IndexGeneratorJob
|
||||||
IncrementalIndex index, Interval interval, File file, ProgressIndicator progressIndicator
|
IncrementalIndex index, Interval interval, File file, ProgressIndicator progressIndicator
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return IndexMerger.persist(index, interval, file, progressIndicator);
|
return IndexMerger.persist(index, interval, file, config.getIndexSpec(), progressIndicator);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -78,7 +79,7 @@ public class LegacyIndexGeneratorJob extends IndexGeneratorJob
|
||||||
ProgressIndicator progressIndicator
|
ProgressIndicator progressIndicator
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return IndexMerger.mergeQueryableIndex(indexes, aggs, file, progressIndicator);
|
return IndexMerger.mergeQueryableIndex(indexes, aggs, file, config.getIndexSpec(), progressIndicator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -173,6 +173,7 @@ public class HadoopDruidIndexerConfigTest
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs),
|
ImmutableMap.of(new DateTime("2010-01-01T01:00:00"), specs),
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
|
|
|
@ -158,7 +158,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
|
|
||||||
fileToUpload = new File(tmpSegmentDir, "merged");
|
fileToUpload = new File(tmpSegmentDir, "merged");
|
||||||
IndexMaker.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload);
|
IndexMaker.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload, config.getIndexSpec());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Map merged segment so we can extract dimensions
|
// Map merged segment so we can extract dimensions
|
||||||
|
@ -205,7 +205,8 @@ public class YeOldePlumberSchool implements PlumberSchool
|
||||||
try {
|
try {
|
||||||
IndexMaker.persist(
|
IndexMaker.persist(
|
||||||
indexToPersist.getIndex(),
|
indexToPersist.getIndex(),
|
||||||
dirToPersist
|
dirToPersist,
|
||||||
|
config.getIndexSpec()
|
||||||
);
|
);
|
||||||
|
|
||||||
indexToPersist.swapSegment(null);
|
indexToPersist.swapSegment(null);
|
||||||
|
|
|
@ -27,6 +27,7 @@ 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.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
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;
|
||||||
|
@ -46,14 +47,19 @@ import java.util.Map;
|
||||||
*/
|
*/
|
||||||
public class AppendTask extends MergeTaskBase
|
public class AppendTask extends MergeTaskBase
|
||||||
{
|
{
|
||||||
|
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public AppendTask(
|
public AppendTask(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("segments") List<DataSegment> segments
|
@JsonProperty("segments") List<DataSegment> segments,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, dataSource, segments);
|
super(id, dataSource, segments);
|
||||||
|
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -120,7 +126,7 @@ public class AppendTask extends MergeTaskBase
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
return IndexMerger.append(adapters, outDir);
|
return IndexMerger.append(adapters, outDir, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -45,6 +45,7 @@ import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.TaskToolbox;
|
import io.druid.indexing.common.TaskToolbox;
|
||||||
import io.druid.indexing.common.index.YeOldePlumberSchool;
|
import io.druid.indexing.common.index.YeOldePlumberSchool;
|
||||||
import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector;
|
import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.IOConfig;
|
import io.druid.segment.indexing.IOConfig;
|
||||||
import io.druid.segment.indexing.IngestionSpec;
|
import io.druid.segment.indexing.IngestionSpec;
|
||||||
|
@ -336,7 +337,20 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
tmpDir
|
tmpDir
|
||||||
).findPlumber(
|
).findPlumber(
|
||||||
schema,
|
schema,
|
||||||
new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec, null, null, null),
|
new RealtimeTuningConfig(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
shardSpec,
|
||||||
|
ingestionSchema.getTuningConfig().getIndexSpec(),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
),
|
||||||
metrics
|
metrics
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -416,7 +430,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
|
|
||||||
this.dataSchema = dataSchema;
|
this.dataSchema = dataSchema;
|
||||||
this.ioConfig = ioConfig;
|
this.ioConfig = ioConfig;
|
||||||
this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null) : tuningConfig;
|
this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null, null) : tuningConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -466,21 +480,25 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
{
|
{
|
||||||
private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000;
|
private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000;
|
||||||
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000;
|
private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000;
|
||||||
|
private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec();
|
||||||
|
|
||||||
private final int targetPartitionSize;
|
private final int targetPartitionSize;
|
||||||
private final int rowFlushBoundary;
|
private final int rowFlushBoundary;
|
||||||
private final int numShards;
|
private final int numShards;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public IndexTuningConfig(
|
public IndexTuningConfig(
|
||||||
@JsonProperty("targetPartitionSize") int targetPartitionSize,
|
@JsonProperty("targetPartitionSize") int targetPartitionSize,
|
||||||
@JsonProperty("rowFlushBoundary") int rowFlushBoundary,
|
@JsonProperty("rowFlushBoundary") int rowFlushBoundary,
|
||||||
@JsonProperty("numShards") @Nullable Integer numShards
|
@JsonProperty("numShards") @Nullable Integer numShards,
|
||||||
|
@JsonProperty("indexSpec") @Nullable IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize;
|
this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize;
|
||||||
this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary;
|
this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary;
|
||||||
this.numShards = numShards == null ? -1 : numShards;
|
this.numShards = numShards == null ? -1 : numShards;
|
||||||
|
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
this.targetPartitionSize == -1 || this.numShards == -1,
|
this.targetPartitionSize == -1 || this.numShards == -1,
|
||||||
"targetPartitionsSize and shardCount both cannot be set"
|
"targetPartitionsSize and shardCount both cannot be set"
|
||||||
|
@ -504,5 +522,11 @@ public class IndexTask extends AbstractFixedIntervalTask
|
||||||
{
|
{
|
||||||
return numShards;
|
return numShards;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public IndexSpec getIndexSpec()
|
||||||
|
{
|
||||||
|
return indexSpec;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ 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.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
@ -41,17 +42,20 @@ public class MergeTask extends MergeTaskBase
|
||||||
{
|
{
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final List<AggregatorFactory> aggregators;
|
private final List<AggregatorFactory> aggregators;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public MergeTask(
|
public MergeTask(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("segments") List<DataSegment> segments,
|
@JsonProperty("segments") List<DataSegment> segments,
|
||||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
@JsonProperty("aggregations") List<AggregatorFactory> aggregators,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, dataSource, segments);
|
super(id, dataSource, segments);
|
||||||
this.aggregators = aggregators;
|
this.aggregators = aggregators;
|
||||||
|
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -76,7 +80,8 @@ public class MergeTask extends MergeTaskBase
|
||||||
}
|
}
|
||||||
),
|
),
|
||||||
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
|
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
|
||||||
outDir
|
outDir,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,6 +31,7 @@ import io.druid.indexing.common.actions.SegmentInsertAction;
|
||||||
import io.druid.indexing.common.actions.SegmentListUsedAction;
|
import io.druid.indexing.common.actions.SegmentListUsedAction;
|
||||||
import io.druid.indexing.common.actions.TaskActionClient;
|
import io.druid.indexing.common.actions.TaskActionClient;
|
||||||
import io.druid.segment.IndexIO;
|
import io.druid.segment.IndexIO;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.loading.SegmentLoadingException;
|
import io.druid.segment.loading.SegmentLoadingException;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -53,11 +54,12 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final DataSegment segment;
|
private final DataSegment segment;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
public static VersionConverterTask create(String dataSource, Interval interval)
|
public static VersionConverterTask create(String dataSource, Interval interval)
|
||||||
{
|
{
|
||||||
final String id = makeId(dataSource, interval);
|
final String id = makeId(dataSource, interval);
|
||||||
return new VersionConverterTask(id, id, dataSource, interval, null);
|
return new VersionConverterTask(id, id, dataSource, interval, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static VersionConverterTask create(DataSegment segment)
|
public static VersionConverterTask create(DataSegment segment)
|
||||||
|
@ -65,7 +67,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
final Interval interval = segment.getInterval();
|
final Interval interval = segment.getInterval();
|
||||||
final String dataSource = segment.getDataSource();
|
final String dataSource = segment.getDataSource();
|
||||||
final String id = makeId(dataSource, interval);
|
final String id = makeId(dataSource, interval);
|
||||||
return new VersionConverterTask(id, id, dataSource, interval, segment);
|
return new VersionConverterTask(id, id, dataSource, interval, segment, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String makeId(String dataSource, Interval interval)
|
private static String makeId(String dataSource, Interval interval)
|
||||||
|
@ -81,7 +83,8 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
@JsonProperty("groupId") String groupId,
|
@JsonProperty("groupId") String groupId,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("interval") Interval interval,
|
@JsonProperty("interval") Interval interval,
|
||||||
@JsonProperty("segment") DataSegment segment
|
@JsonProperty("segment") DataSegment segment,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (id == null) {
|
if (id == null) {
|
||||||
|
@ -91,7 +94,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
return create(segment);
|
return create(segment);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new VersionConverterTask(id, groupId, dataSource, interval, segment);
|
return new VersionConverterTask(id, groupId, dataSource, interval, segment, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
private VersionConverterTask(
|
private VersionConverterTask(
|
||||||
|
@ -99,11 +102,13 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
String groupId,
|
String groupId,
|
||||||
String dataSource,
|
String dataSource,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
DataSegment segment
|
DataSegment segment,
|
||||||
|
IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, groupId, dataSource, interval);
|
super(id, groupId, dataSource, interval);
|
||||||
this.segment = segment;
|
this.segment = segment;
|
||||||
|
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -138,7 +143,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
{
|
{
|
||||||
final Integer segmentVersion = segment.getBinaryVersion();
|
final Integer segmentVersion = segment.getBinaryVersion();
|
||||||
if (!CURR_VERSION_INTEGER.equals(segmentVersion)) {
|
if (!CURR_VERSION_INTEGER.equals(segmentVersion)) {
|
||||||
return new SubTask(getGroupId(), segment);
|
return new SubTask(getGroupId(), segment, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion);
|
log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion);
|
||||||
|
@ -156,7 +161,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
log.info("I'm in a subless mood.");
|
log.info("I'm in a subless mood.");
|
||||||
convertSegment(toolbox, segment);
|
convertSegment(toolbox, segment, indexSpec);
|
||||||
}
|
}
|
||||||
return success();
|
return success();
|
||||||
}
|
}
|
||||||
|
@ -184,11 +189,13 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
{
|
{
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final DataSegment segment;
|
private final DataSegment segment;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public SubTask(
|
public SubTask(
|
||||||
@JsonProperty("groupId") String groupId,
|
@JsonProperty("groupId") String groupId,
|
||||||
@JsonProperty("segment") DataSegment segment
|
@JsonProperty("segment") DataSegment segment,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
|
@ -204,6 +211,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
segment.getInterval()
|
segment.getInterval()
|
||||||
);
|
);
|
||||||
this.segment = segment;
|
this.segment = segment;
|
||||||
|
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -222,12 +230,12 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
log.info("Subs are good! Italian BMT and Meatball are probably my favorite.");
|
log.info("Subs are good! Italian BMT and Meatball are probably my favorite.");
|
||||||
convertSegment(toolbox, segment);
|
convertSegment(toolbox, segment, indexSpec);
|
||||||
return success();
|
return success();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void convertSegment(TaskToolbox toolbox, final DataSegment segment)
|
private static void convertSegment(TaskToolbox toolbox, final DataSegment segment, IndexSpec indexSpec)
|
||||||
throws SegmentLoadingException, IOException
|
throws SegmentLoadingException, IOException
|
||||||
{
|
{
|
||||||
log.info("Converting segment[%s]", segment);
|
log.info("Converting segment[%s]", segment);
|
||||||
|
@ -250,7 +258,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask
|
||||||
|
|
||||||
final File location = localSegments.get(segment);
|
final File location = localSegments.get(segment);
|
||||||
final File outLocation = new File(location, "v9_out");
|
final File outLocation = new File(location, "v9_out");
|
||||||
if (IndexIO.convertSegment(location, outLocation)) {
|
if (IndexIO.convertSegment(location, outLocation, indexSpec)) {
|
||||||
final int outVersion = IndexIO.getVersionFromDir(outLocation);
|
final int outVersion = IndexIO.getVersionFromDir(outLocation);
|
||||||
|
|
||||||
// Appending to the version makes a new version that inherits most comparability parameters of the original
|
// Appending to the version makes a new version that inherits most comparability parameters of the original
|
||||||
|
|
|
@ -29,7 +29,6 @@ import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.google.inject.Provider;
|
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.common.guava.Sequences;
|
import com.metamx.common.guava.Sequences;
|
||||||
import com.metamx.common.guava.Yielder;
|
import com.metamx.common.guava.Yielder;
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.indexing.common.task.MergeTask;
|
import io.druid.indexing.common.task.MergeTask;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -52,7 +53,8 @@ public class TestMergeTask extends MergeTask
|
||||||
0
|
0
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
Lists.<AggregatorFactory>newArrayList()
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -63,10 +65,11 @@ public class TestMergeTask extends MergeTask
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("segments") List<DataSegment> segments,
|
@JsonProperty("segments") List<DataSegment> segments,
|
||||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
@JsonProperty("aggregations") List<AggregatorFactory> aggregators,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, dataSource, segments, aggregators);
|
super(id, dataSource, segments, aggregators, indexSpec);
|
||||||
this.id = id;
|
this.id = id;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,6 +35,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
|
||||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
@ -54,6 +55,8 @@ import java.util.List;
|
||||||
|
|
||||||
public class IndexTaskTest
|
public class IndexTaskTest
|
||||||
{
|
{
|
||||||
|
private final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDeterminePartitions() throws Exception
|
public void testDeterminePartitions() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -108,7 +111,8 @@ public class IndexTaskTest
|
||||||
new IndexTask.IndexTuningConfig(
|
new IndexTask.IndexTuningConfig(
|
||||||
2,
|
2,
|
||||||
0,
|
0,
|
||||||
null
|
null,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new DefaultObjectMapper()
|
new DefaultObjectMapper()
|
||||||
|
|
|
@ -31,6 +31,7 @@ import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.RealtimeIOConfig;
|
import io.druid.segment.indexing.RealtimeIOConfig;
|
||||||
import io.druid.segment.indexing.RealtimeTuningConfig;
|
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||||
|
@ -53,6 +54,8 @@ public class TaskSerdeTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||||
|
|
||||||
|
private final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIndexTaskSerde() throws Exception
|
public void testIndexTaskSerde() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -70,7 +73,7 @@ public class TaskSerdeTest
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)),
|
new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)),
|
||||||
new IndexTask.IndexTuningConfig(10000, -1, -1)
|
new IndexTask.IndexTuningConfig(10000, -1, -1, indexSpec)
|
||||||
),
|
),
|
||||||
jsonMapper
|
jsonMapper
|
||||||
);
|
);
|
||||||
|
@ -107,7 +110,8 @@ public class TaskSerdeTest
|
||||||
),
|
),
|
||||||
ImmutableList.<AggregatorFactory>of(
|
ImmutableList.<AggregatorFactory>of(
|
||||||
new CountAggregatorFactory("cnt")
|
new CountAggregatorFactory("cnt")
|
||||||
)
|
),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
final String json = jsonMapper.writeValueAsString(task);
|
final String json = jsonMapper.writeValueAsString(task);
|
||||||
|
@ -179,7 +183,8 @@ public class TaskSerdeTest
|
||||||
{
|
{
|
||||||
final VersionConverterTask.SubTask task = new VersionConverterTask.SubTask(
|
final VersionConverterTask.SubTask task = new VersionConverterTask.SubTask(
|
||||||
"myGroupId",
|
"myGroupId",
|
||||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build(),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
final String json = jsonMapper.writeValueAsString(task);
|
final String json = jsonMapper.writeValueAsString(task);
|
||||||
|
@ -229,6 +234,7 @@ public class TaskSerdeTest
|
||||||
null,
|
null,
|
||||||
1,
|
1,
|
||||||
new NoneShardSpec(),
|
new NoneShardSpec(),
|
||||||
|
indexSpec,
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
null
|
null
|
||||||
|
@ -277,7 +283,8 @@ public class TaskSerdeTest
|
||||||
"foo",
|
"foo",
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||||
)
|
),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
final String json = jsonMapper.writeValueAsString(task);
|
final String json = jsonMapper.writeValueAsString(task);
|
||||||
|
|
|
@ -61,6 +61,7 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
import io.druid.query.filter.SelectorDimFilter;
|
import io.druid.query.filter.SelectorDimFilter;
|
||||||
import io.druid.segment.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
import io.druid.segment.loading.DataSegmentArchiver;
|
import io.druid.segment.loading.DataSegmentArchiver;
|
||||||
|
@ -100,9 +101,11 @@ import java.util.Set;
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class IngestSegmentFirehoseFactoryTest
|
public class IngestSegmentFirehoseFactoryTest
|
||||||
{
|
{
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "{1}")
|
@Parameterized.Parameters(name = "{1}")
|
||||||
public static Collection<Object[]> constructorFeeder() throws IOException
|
public static Collection<Object[]> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
|
final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
final HeapMemoryTaskStorage ts = new HeapMemoryTaskStorage(
|
final HeapMemoryTaskStorage ts = new HeapMemoryTaskStorage(
|
||||||
new TaskStorageConfig(null)
|
new TaskStorageConfig(null)
|
||||||
|
@ -132,7 +135,7 @@ public class IngestSegmentFirehoseFactoryTest
|
||||||
if (!persistDir.mkdirs() && !persistDir.exists()) {
|
if (!persistDir.mkdirs() && !persistDir.exists()) {
|
||||||
throw new IOException(String.format("Could not create directory at [%s]", persistDir.getAbsolutePath()));
|
throw new IOException(String.format("Could not create directory at [%s]", persistDir.getAbsolutePath()));
|
||||||
}
|
}
|
||||||
IndexMerger.persist(index, persistDir);
|
IndexMerger.persist(index, persistDir, indexSpec);
|
||||||
|
|
||||||
final TaskLockbox tl = new TaskLockbox(ts);
|
final TaskLockbox tl = new TaskLockbox(ts);
|
||||||
final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null)
|
final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null)
|
||||||
|
|
|
@ -63,6 +63,7 @@ import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.segment.indexing.DataSchema;
|
import io.druid.segment.indexing.DataSchema;
|
||||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import io.druid.segment.loading.DataSegmentArchiver;
|
import io.druid.segment.loading.DataSegmentArchiver;
|
||||||
|
@ -109,6 +110,7 @@ public class TaskLifecycleTest
|
||||||
private MockIndexerMetadataStorageCoordinator mdc = null;
|
private MockIndexerMetadataStorageCoordinator mdc = null;
|
||||||
private TaskActionClientFactory tac = null;
|
private TaskActionClientFactory tac = null;
|
||||||
private TaskToolboxFactory tb = null;
|
private TaskToolboxFactory tb = null;
|
||||||
|
private IndexSpec indexSpec;
|
||||||
|
|
||||||
private static MockIndexerMetadataStorageCoordinator newMockMDC()
|
private static MockIndexerMetadataStorageCoordinator newMockMDC()
|
||||||
{
|
{
|
||||||
|
@ -248,6 +250,7 @@ public class TaskLifecycleTest
|
||||||
"{\"startDelay\":\"PT0S\", \"restartDelay\":\"PT1S\"}",
|
"{\"startDelay\":\"PT0S\", \"restartDelay\":\"PT1S\"}",
|
||||||
TaskQueueConfig.class
|
TaskQueueConfig.class
|
||||||
);
|
);
|
||||||
|
indexSpec = new IndexSpec();
|
||||||
ts = new HeapMemoryTaskStorage(
|
ts = new HeapMemoryTaskStorage(
|
||||||
new TaskStorageConfig(null)
|
new TaskStorageConfig(null)
|
||||||
{
|
{
|
||||||
|
@ -361,7 +364,7 @@ public class TaskLifecycleTest
|
||||||
IR("2010-01-02T01", "a", "c", 1)
|
IR("2010-01-02T01", "a", "c", 1)
|
||||||
)
|
)
|
||||||
)),
|
)),
|
||||||
new IndexTask.IndexTuningConfig(10000, -1, -1)),
|
new IndexTask.IndexTuningConfig(10000, -1, -1, indexSpec)),
|
||||||
TestUtils.MAPPER
|
TestUtils.MAPPER
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -415,7 +418,7 @@ public class TaskLifecycleTest
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new IndexTask.IndexIOConfig(newMockExceptionalFirehoseFactory()),
|
new IndexTask.IndexIOConfig(newMockExceptionalFirehoseFactory()),
|
||||||
new IndexTask.IndexTuningConfig(10000, -1, -1)
|
new IndexTask.IndexTuningConfig(10000, -1, -1, indexSpec)
|
||||||
),
|
),
|
||||||
TestUtils.MAPPER
|
TestUtils.MAPPER
|
||||||
);
|
);
|
||||||
|
|
|
@ -33,7 +33,8 @@ public class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig
|
||||||
@Override
|
@Override
|
||||||
public long getMaxZnodeBytes()
|
public long getMaxZnodeBytes()
|
||||||
{
|
{
|
||||||
return 1000;
|
// make sure this is large enough, otherwise RemoteTaskRunnerTest might fail unexpectedly
|
||||||
|
return 10 * 1024;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -35,6 +35,7 @@ import io.druid.indexing.worker.TaskAnnouncement;
|
||||||
import io.druid.indexing.worker.Worker;
|
import io.druid.indexing.worker.Worker;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
|
@ -64,6 +65,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
{
|
{
|
||||||
autoScaler = EasyMock.createMock(AutoScaler.class);
|
autoScaler = EasyMock.createMock(AutoScaler.class);
|
||||||
|
|
||||||
|
final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
testTask = new TestMergeTask(
|
testTask = new TestMergeTask(
|
||||||
"task1",
|
"task1",
|
||||||
"dummyDs",
|
"dummyDs",
|
||||||
|
@ -80,7 +83,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
0
|
0
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
Lists.<AggregatorFactory>newArrayList()
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
simpleResourceManagementConfig = new SimpleResourceManagementConfig()
|
simpleResourceManagementConfig = new SimpleResourceManagementConfig()
|
||||||
|
|
|
@ -20,6 +20,7 @@ package io.druid.segment;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
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.google.common.base.Suppliers;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
|
@ -61,8 +62,12 @@ import io.druid.segment.data.BitmapSerde;
|
||||||
import io.druid.segment.data.BitmapSerdeFactory;
|
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.CompressedObjectStrategy;
|
||||||
|
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.IndexedIterable;
|
import io.druid.segment.data.IndexedIterable;
|
||||||
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
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;
|
||||||
|
@ -121,10 +126,12 @@ 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;
|
||||||
|
|
||||||
|
@Deprecated // specify bitmap type in IndexSpec instead
|
||||||
|
protected static final BitmapSerdeFactory CONFIGURED_BITMAP_SERDE_FACTORY;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
|
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
|
||||||
ImmutableList.<Module>of(
|
ImmutableList.<Module>of(
|
||||||
|
@ -140,6 +147,7 @@ public class IndexIO
|
||||||
);
|
);
|
||||||
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
||||||
|
|
||||||
|
// this property is deprecated, use IndexSpec instead
|
||||||
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
|
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -147,7 +155,7 @@ public class IndexIO
|
||||||
);
|
);
|
||||||
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);
|
CONFIGURED_BITMAP_SERDE_FACTORY = injector.getInstance(BitmapSerdeFactory.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static QueryableIndex loadIndex(File inDir) throws IOException
|
public static QueryableIndex loadIndex(File inDir) throws IOException
|
||||||
|
@ -186,7 +194,7 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static boolean convertSegment(File toConvert, File converted) throws IOException
|
public static boolean convertSegment(File toConvert, File converted, IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
final int version = SegmentUtils.getVersionFromDir(toConvert);
|
final int version = SegmentUtils.getVersionFromDir(toConvert);
|
||||||
|
|
||||||
|
@ -205,11 +213,12 @@ public class IndexIO
|
||||||
log.info("Old version, re-persisting.");
|
log.info("Old version, re-persisting.");
|
||||||
IndexMerger.append(
|
IndexMerger.append(
|
||||||
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
|
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
|
||||||
converted
|
converted,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
return true;
|
return true;
|
||||||
case 8:
|
case 8:
|
||||||
DefaultIndexIOHandler.convertV8toV9(toConvert, converted);
|
DefaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec);
|
||||||
return true;
|
return true;
|
||||||
default:
|
default:
|
||||||
log.info("Version[%s], skipping.", version);
|
log.info("Version[%s], skipping.", version);
|
||||||
|
@ -328,7 +337,7 @@ public class IndexIO
|
||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void convertV8toV9(File v8Dir, File v9Dir) throws IOException
|
public static void convertV8toV9(File v8Dir, File v9Dir, IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir);
|
log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir);
|
||||||
|
|
||||||
|
@ -353,6 +362,8 @@ public class IndexIO
|
||||||
|
|
||||||
Map<String, GenericIndexed<ImmutableBitmap>> bitmapIndexes = Maps.newHashMap();
|
Map<String, GenericIndexed<ImmutableBitmap>> bitmapIndexes = Maps.newHashMap();
|
||||||
final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd");
|
final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd");
|
||||||
|
BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
||||||
|
|
||||||
while (invertedBuffer.hasRemaining()) {
|
while (invertedBuffer.hasRemaining()) {
|
||||||
final String dimName = serializerUtils.readString(invertedBuffer);
|
final String dimName = serializerUtils.readString(invertedBuffer);
|
||||||
bitmapIndexes.put(
|
bitmapIndexes.put(
|
||||||
|
@ -404,7 +415,7 @@ public class IndexIO
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
VSizeIndexedInts singleValCol = null;
|
List<Integer> singleValCol = null;
|
||||||
VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer());
|
VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer());
|
||||||
GenericIndexed<ImmutableBitmap> bitmaps = bitmapIndexes.get(dimension);
|
GenericIndexed<ImmutableBitmap> bitmaps = bitmapIndexes.get(dimension);
|
||||||
ImmutableRTree spatialIndex = spatialIndexes.get(dimension);
|
ImmutableRTree spatialIndex = spatialIndexes.get(dimension);
|
||||||
|
@ -468,41 +479,61 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
|
|
||||||
final VSizeIndexed finalMultiValCol = multiValCol;
|
final VSizeIndexed finalMultiValCol = multiValCol;
|
||||||
singleValCol = VSizeIndexedInts.fromList(
|
singleValCol = new AbstractList<Integer>()
|
||||||
new AbstractList<Integer>()
|
{
|
||||||
{
|
@Override
|
||||||
@Override
|
public Integer get(int index)
|
||||||
public Integer get(int index)
|
{
|
||||||
{
|
final VSizeIndexedInts ints = finalMultiValCol.get(index);
|
||||||
final VSizeIndexedInts ints = finalMultiValCol.get(index);
|
return ints.size() == 0 ? 0 : ints.get(0) + (bumpedDictionary ? 1 : 0);
|
||||||
return ints.size() == 0 ? 0 : ints.get(0) + (bumpedDictionary ? 1 : 0);
|
}
|
||||||
}
|
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return finalMultiValCol.size();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
@Override
|
|
||||||
public int size()
|
|
||||||
{
|
|
||||||
return finalMultiValCol.size();
|
|
||||||
}
|
|
||||||
},
|
|
||||||
dictionary.size()
|
|
||||||
);
|
|
||||||
multiValCol = null;
|
multiValCol = null;
|
||||||
} else {
|
} else {
|
||||||
builder.setHasMultipleValues(true);
|
builder.setHasMultipleValues(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
builder.addSerde(
|
final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy();
|
||||||
new DictionaryEncodedColumnPartSerde(
|
|
||||||
dictionary,
|
|
||||||
singleValCol,
|
|
||||||
multiValCol,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
bitmaps,
|
|
||||||
spatialIndex
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
final ColumnDescriptor serdeficator = builder.build();
|
final DictionaryEncodedColumnPartSerde.Builder columnPartBuilder = DictionaryEncodedColumnPartSerde
|
||||||
|
.builder()
|
||||||
|
.withDictionary(dictionary)
|
||||||
|
.withBitmapSerdeFactory(bitmapSerdeFactory)
|
||||||
|
.withBitmaps(bitmaps)
|
||||||
|
.withSpatialIndex(spatialIndex)
|
||||||
|
.withByteOrder(BYTE_ORDER);
|
||||||
|
|
||||||
|
if (singleValCol != null) {
|
||||||
|
if (compressionStrategy != null) {
|
||||||
|
columnPartBuilder.withSingleValuedColumn(
|
||||||
|
CompressedVSizeIntsIndexedSupplier.fromList(
|
||||||
|
singleValCol,
|
||||||
|
dictionary.size(),
|
||||||
|
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(dictionary.size()),
|
||||||
|
BYTE_ORDER,
|
||||||
|
compressionStrategy
|
||||||
|
)
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
columnPartBuilder.withSingleValuedColumn(VSizeIndexedInts.fromList(singleValCol, dictionary.size()));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if(compressionStrategy != null) {
|
||||||
|
log.info("Compression not supported for multi-value dimensions, defaulting to `uncompressed` for dimension[%s]", dimension);
|
||||||
|
}
|
||||||
|
columnPartBuilder.withMultiValuedColumn(multiValCol);
|
||||||
|
}
|
||||||
|
|
||||||
|
final ColumnDescriptor serdeficator = builder
|
||||||
|
.addSerde(columnPartBuilder.build())
|
||||||
|
.build();
|
||||||
|
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
|
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
|
||||||
|
@ -663,7 +694,9 @@ public class IndexIO
|
||||||
new DictionaryEncodedColumnSupplier(
|
new DictionaryEncodedColumnSupplier(
|
||||||
index.getDimValueLookup(dimension),
|
index.getDimValueLookup(dimension),
|
||||||
null,
|
null,
|
||||||
index.getDimColumn(dimension),
|
Suppliers.<IndexedMultivalue<IndexedInts>>ofInstance(
|
||||||
|
index.getDimColumn(dimension)
|
||||||
|
),
|
||||||
columnConfig.columnCacheSizeBytes()
|
columnConfig.columnCacheSizeBytes()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
|
@ -32,7 +32,6 @@ 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.google.inject.Module;
|
||||||
import com.metamx.collections.bitmap.BitmapFactory;
|
import com.metamx.collections.bitmap.BitmapFactory;
|
||||||
|
@ -53,7 +52,6 @@ import io.druid.collections.CombiningIterable;
|
||||||
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.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
|
@ -63,6 +61,7 @@ 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.CompressedVSizeIntsIndexedSupplier;
|
||||||
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;
|
||||||
|
@ -108,28 +107,15 @@ 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.makeStartupInjectorWithModules(
|
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.<Module>of());
|
||||||
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, final IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
return persist(index, index.getInterval(), outDir);
|
return persist(index, index.getInterval(), outDir, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -145,16 +131,18 @@ public class IndexMaker
|
||||||
public static File persist(
|
public static File persist(
|
||||||
final IncrementalIndex index,
|
final IncrementalIndex index,
|
||||||
final Interval dataInterval,
|
final Interval dataInterval,
|
||||||
File outDir
|
File outDir,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return persist(index, dataInterval, outDir, new LoggingProgressIndicator(outDir.toString()));
|
return persist(index, dataInterval, outDir, indexSpec, new LoggingProgressIndicator(outDir.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File persist(
|
public static File persist(
|
||||||
final IncrementalIndex index,
|
final IncrementalIndex index,
|
||||||
final Interval dataInterval,
|
final Interval dataInterval,
|
||||||
File outDir,
|
File outDir,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
ProgressIndicator progress
|
ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -186,26 +174,28 @@ public class IndexMaker
|
||||||
new IncrementalIndexAdapter(
|
new IncrementalIndexAdapter(
|
||||||
dataInterval,
|
dataInterval,
|
||||||
index,
|
index,
|
||||||
bitmapSerdeFactory.getBitmapFactory()
|
indexSpec.getBitmapSerdeFactory().getBitmapFactory()
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
index.getMetricAggs(),
|
index.getMetricAggs(),
|
||||||
outDir,
|
outDir,
|
||||||
|
indexSpec,
|
||||||
progress
|
progress
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File mergeQueryableIndex(
|
public static File mergeQueryableIndex(
|
||||||
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir
|
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return mergeQueryableIndex(indexes, metricAggs, outDir, new LoggingProgressIndicator(outDir.toString()));
|
return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new LoggingProgressIndicator(outDir.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File mergeQueryableIndex(
|
public static File mergeQueryableIndex(
|
||||||
List<QueryableIndex> indexes,
|
List<QueryableIndex> indexes,
|
||||||
final AggregatorFactory[] metricAggs,
|
final AggregatorFactory[] metricAggs,
|
||||||
File outDir,
|
File outDir,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
ProgressIndicator progress
|
ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -223,21 +213,23 @@ public class IndexMaker
|
||||||
),
|
),
|
||||||
metricAggs,
|
metricAggs,
|
||||||
outDir,
|
outDir,
|
||||||
|
indexSpec,
|
||||||
progress
|
progress
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File merge(
|
public static File merge(
|
||||||
List<IndexableAdapter> adapters, final AggregatorFactory[] metricAggs, File outDir
|
List<IndexableAdapter> adapters, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return merge(adapters, metricAggs, outDir, new LoggingProgressIndicator(outDir.toString()));
|
return merge(adapters, metricAggs, outDir, indexSpec, new LoggingProgressIndicator(outDir.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File merge(
|
public static File merge(
|
||||||
List<IndexableAdapter> adapters,
|
List<IndexableAdapter> adapters,
|
||||||
final AggregatorFactory[] metricAggs,
|
final AggregatorFactory[] metricAggs,
|
||||||
File outDir,
|
File outDir,
|
||||||
|
final IndexSpec indexSpec,
|
||||||
ProgressIndicator progress
|
ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -326,21 +318,23 @@ public class IndexMaker
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn);
|
return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn, indexSpec);
|
||||||
}
|
|
||||||
|
|
||||||
public static File append(
|
|
||||||
final List<IndexableAdapter> adapters,
|
|
||||||
File outDir
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
return append(adapters, outDir, new LoggingProgressIndicator(outDir.toString()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File append(
|
public static File append(
|
||||||
final List<IndexableAdapter> adapters,
|
final List<IndexableAdapter> adapters,
|
||||||
final File outDir,
|
final File outDir,
|
||||||
final ProgressIndicator progress
|
final IndexSpec indexSpec
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
return append(adapters, outDir, new LoggingProgressIndicator(outDir.toString()), indexSpec);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static File append(
|
||||||
|
final List<IndexableAdapter> adapters,
|
||||||
|
final File outDir,
|
||||||
|
final ProgressIndicator progress,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
FileUtils.deleteDirectory(outDir);
|
FileUtils.deleteDirectory(outDir);
|
||||||
|
@ -409,7 +403,7 @@ public class IndexMaker
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn);
|
return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static File makeIndexFiles(
|
private static File makeIndexFiles(
|
||||||
|
@ -418,7 +412,8 @@ public class IndexMaker
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn
|
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
progress.start();
|
progress.start();
|
||||||
|
@ -501,14 +496,15 @@ public class IndexMaker
|
||||||
theRows,
|
theRows,
|
||||||
columnCapabilities,
|
columnCapabilities,
|
||||||
dimensionValuesLookup,
|
dimensionValuesLookup,
|
||||||
rowNumConversions
|
rowNumConversions,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
|
|
||||||
progress.progress();
|
progress.progress();
|
||||||
makeMetricColumns(v9Smoosher, progress, theRows, mergedMetrics, valueTypes, metricTypeNames, rowCount);
|
makeMetricColumns(v9Smoosher, progress, theRows, mergedMetrics, valueTypes, metricTypeNames, rowCount, indexSpec);
|
||||||
|
|
||||||
progress.progress();
|
progress.progress();
|
||||||
makeIndexBinary(v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, skippedDimensions, progress);
|
makeIndexBinary(v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, skippedDimensions, progress, indexSpec);
|
||||||
|
|
||||||
v9Smoosher.close();
|
v9Smoosher.close();
|
||||||
|
|
||||||
|
@ -768,7 +764,8 @@ public class IndexMaker
|
||||||
final Iterable<Rowboat> theRows,
|
final Iterable<Rowboat> theRows,
|
||||||
final Map<String, ColumnCapabilitiesImpl> columnCapabilities,
|
final Map<String, ColumnCapabilitiesImpl> columnCapabilities,
|
||||||
final Map<String, Iterable<String>> dimensionValuesLookup,
|
final Map<String, Iterable<String>> dimensionValuesLookup,
|
||||||
final List<IntBuffer> rowNumConversions
|
final List<IntBuffer> rowNumConversions,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String dimSection = "make dimension columns";
|
final String dimSection = "make dimension columns";
|
||||||
|
@ -790,7 +787,9 @@ public class IndexMaker
|
||||||
dimension,
|
dimension,
|
||||||
columnCapabilities,
|
columnCapabilities,
|
||||||
dimensionValuesLookup,
|
dimensionValuesLookup,
|
||||||
rowNumConversions
|
rowNumConversions,
|
||||||
|
indexSpec.getBitmapSerdeFactory(),
|
||||||
|
indexSpec.getDimensionCompressionStrategy()
|
||||||
);
|
);
|
||||||
dimIndex++;
|
dimIndex++;
|
||||||
}
|
}
|
||||||
|
@ -806,7 +805,9 @@ public class IndexMaker
|
||||||
final String dimension,
|
final String dimension,
|
||||||
final Map<String, ColumnCapabilitiesImpl> columnCapabilities,
|
final Map<String, ColumnCapabilitiesImpl> columnCapabilities,
|
||||||
final Map<String, Iterable<String>> dimensionValuesLookup,
|
final Map<String, Iterable<String>> dimensionValuesLookup,
|
||||||
final List<IntBuffer> rowNumConversions
|
final List<IntBuffer> rowNumConversions,
|
||||||
|
final BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
|
final CompressedObjectStrategy.CompressionStrategy compressionStrategy
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = String.format("make %s", dimension);
|
final String section = String.format("make %s", dimension);
|
||||||
|
@ -825,8 +826,8 @@ public class IndexMaker
|
||||||
dimBuilder.setHasMultipleValues(hasMultipleValues);
|
dimBuilder.setHasMultipleValues(hasMultipleValues);
|
||||||
|
|
||||||
// make dimension columns
|
// make dimension columns
|
||||||
VSizeIndexedInts singleValCol = null;
|
List<Integer> singleValCol;
|
||||||
VSizeIndexed multiValCol = null;
|
final VSizeIndexed multiValCol;
|
||||||
|
|
||||||
ColumnDictionaryEntryStore adder = hasMultipleValues
|
ColumnDictionaryEntryStore adder = hasMultipleValues
|
||||||
? new MultiValColumnDictionaryEntryStore()
|
? new MultiValColumnDictionaryEntryStore()
|
||||||
|
@ -881,6 +882,8 @@ public class IndexMaker
|
||||||
);
|
);
|
||||||
|
|
||||||
final int dictionarySize = dictionary.size();
|
final int dictionarySize = dictionary.size();
|
||||||
|
|
||||||
|
singleValCol = null;
|
||||||
multiValCol = VSizeIndexed.fromIterable(
|
multiValCol = VSizeIndexed.fromIterable(
|
||||||
FunctionalIterable
|
FunctionalIterable
|
||||||
.create(vals)
|
.create(vals)
|
||||||
|
@ -935,6 +938,7 @@ public class IndexMaker
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
final int dictionarySize = dictionary.size();
|
final int dictionarySize = dictionary.size();
|
||||||
|
singleValCol = null;
|
||||||
multiValCol = VSizeIndexed.fromIterable(
|
multiValCol = VSizeIndexed.fromIterable(
|
||||||
FunctionalIterable
|
FunctionalIterable
|
||||||
.create(vals)
|
.create(vals)
|
||||||
|
@ -973,6 +977,7 @@ public class IndexMaker
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
final int dictionarySize = dictionary.size();
|
final int dictionarySize = dictionary.size();
|
||||||
|
singleValCol = null;
|
||||||
multiValCol = VSizeIndexed.fromIterable(
|
multiValCol = VSizeIndexed.fromIterable(
|
||||||
FunctionalIterable
|
FunctionalIterable
|
||||||
.create(vals)
|
.create(vals)
|
||||||
|
@ -1008,50 +1013,62 @@ public class IndexMaker
|
||||||
Iterables.concat(nullList, dimensionValues),
|
Iterables.concat(nullList, dimensionValues),
|
||||||
GenericIndexed.stringStrategy
|
GenericIndexed.stringStrategy
|
||||||
);
|
);
|
||||||
singleValCol = VSizeIndexedInts.fromList(
|
multiValCol = null;
|
||||||
new AbstractList<Integer>()
|
singleValCol = new AbstractList<Integer>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Integer get(int index)
|
public Integer get(int index)
|
||||||
{
|
{
|
||||||
Integer val = vals.get(index);
|
Integer val = vals.get(index);
|
||||||
if (val == null) {
|
if (val == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return val + 1;
|
return val + 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return vals.size();
|
return vals.size();
|
||||||
}
|
}
|
||||||
}, dictionary.size()
|
};
|
||||||
);
|
|
||||||
} else {
|
} else {
|
||||||
singleValCol = VSizeIndexedInts.fromList(
|
multiValCol = null;
|
||||||
new AbstractList<Integer>()
|
singleValCol = new AbstractList<Integer>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Integer get(int index)
|
public Integer get(int index)
|
||||||
{
|
{
|
||||||
Integer val = vals.get(index);
|
Integer val = vals.get(index);
|
||||||
if (val == null) {
|
if (val == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return val;
|
return val;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int size()
|
public int size()
|
||||||
{
|
{
|
||||||
return vals.size();
|
return vals.size();
|
||||||
}
|
}
|
||||||
}, dictionary.size()
|
};
|
||||||
);
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
singleValCol = VSizeIndexedInts.fromList(vals, dictionary.size());
|
multiValCol = null;
|
||||||
|
singleValCol = new AbstractList<Integer>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Integer get(int index)
|
||||||
|
{
|
||||||
|
return vals.get(index);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return vals.size();
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1174,16 +1191,38 @@ public class IndexMaker
|
||||||
|
|
||||||
log.info("Completed dimension[%s] with cardinality[%,d]. Starting write.", dimension, dictionary.size());
|
log.info("Completed dimension[%s] with cardinality[%,d]. Starting write.", dimension, dictionary.size());
|
||||||
|
|
||||||
|
final DictionaryEncodedColumnPartSerde.Builder dimPartBuilder = DictionaryEncodedColumnPartSerde
|
||||||
|
.builder()
|
||||||
|
.withDictionary(dictionary)
|
||||||
|
.withBitmapSerdeFactory(bitmapSerdeFactory)
|
||||||
|
.withBitmaps(bitmaps)
|
||||||
|
.withSpatialIndex(spatialIndex)
|
||||||
|
.withByteOrder(IndexIO.BYTE_ORDER);
|
||||||
|
|
||||||
|
if (singleValCol != null) {
|
||||||
|
if (compressionStrategy != null) {
|
||||||
|
dimPartBuilder.withSingleValuedColumn(
|
||||||
|
CompressedVSizeIntsIndexedSupplier.fromList(
|
||||||
|
singleValCol,
|
||||||
|
dictionary.size(),
|
||||||
|
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(dictionary.size()),
|
||||||
|
IndexIO.BYTE_ORDER,
|
||||||
|
compressionStrategy
|
||||||
|
)
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
dimPartBuilder.withSingleValuedColumn(VSizeIndexedInts.fromList(singleValCol, dictionary.size()));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if(compressionStrategy != null) {
|
||||||
|
log.info("Compression not supported for multi-value dimensions, defaulting to `uncompressed` for dimension[%s]", dimension);
|
||||||
|
}
|
||||||
|
dimPartBuilder.withMultiValuedColumn(multiValCol);
|
||||||
|
}
|
||||||
|
|
||||||
writeColumn(
|
writeColumn(
|
||||||
v9Smoosher,
|
v9Smoosher,
|
||||||
new DictionaryEncodedColumnPartSerde(
|
dimPartBuilder.build(),
|
||||||
dictionary,
|
|
||||||
singleValCol,
|
|
||||||
multiValCol,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
bitmaps,
|
|
||||||
spatialIndex
|
|
||||||
),
|
|
||||||
dimBuilder,
|
dimBuilder,
|
||||||
dimension
|
dimension
|
||||||
);
|
);
|
||||||
|
@ -1198,7 +1237,8 @@ public class IndexMaker
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final Map<String, ValueType> valueTypes,
|
final Map<String, ValueType> valueTypes,
|
||||||
final Map<String, String> metricTypeNames,
|
final Map<String, String> metricTypeNames,
|
||||||
final int rowCount
|
final int rowCount,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String metSection = "make metric columns";
|
final String metSection = "make metric columns";
|
||||||
|
@ -1206,7 +1246,17 @@ public class IndexMaker
|
||||||
|
|
||||||
int metIndex = 0;
|
int metIndex = 0;
|
||||||
for (String metric : mergedMetrics) {
|
for (String metric : mergedMetrics) {
|
||||||
makeMetricColumn(v9Smoosher, progress, theRows, metIndex, metric, valueTypes, metricTypeNames, rowCount);
|
makeMetricColumn(
|
||||||
|
v9Smoosher,
|
||||||
|
progress,
|
||||||
|
theRows,
|
||||||
|
metIndex,
|
||||||
|
metric,
|
||||||
|
valueTypes,
|
||||||
|
metricTypeNames,
|
||||||
|
rowCount,
|
||||||
|
indexSpec.getMetricCompressionStrategy()
|
||||||
|
);
|
||||||
metIndex++;
|
metIndex++;
|
||||||
}
|
}
|
||||||
progress.stopSection(metSection);
|
progress.stopSection(metSection);
|
||||||
|
@ -1220,7 +1270,8 @@ public class IndexMaker
|
||||||
final String metric,
|
final String metric,
|
||||||
final Map<String, ValueType> valueTypes,
|
final Map<String, ValueType> valueTypes,
|
||||||
final Map<String, String> metricTypeNames,
|
final Map<String, String> metricTypeNames,
|
||||||
final int rowCount
|
final int rowCount,
|
||||||
|
final CompressedObjectStrategy.CompressionStrategy compressionStrategy
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = String.format("make column[%s]", metric);
|
final String section = String.format("make column[%s]", metric);
|
||||||
|
@ -1243,7 +1294,7 @@ public class IndexMaker
|
||||||
CompressedFloatsIndexedSupplier compressedFloats = CompressedFloatsIndexedSupplier.fromFloatBuffer(
|
CompressedFloatsIndexedSupplier compressedFloats = CompressedFloatsIndexedSupplier.fromFloatBuffer(
|
||||||
FloatBuffer.wrap(arr),
|
FloatBuffer.wrap(arr),
|
||||||
IndexIO.BYTE_ORDER,
|
IndexIO.BYTE_ORDER,
|
||||||
CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
|
compressionStrategy
|
||||||
);
|
);
|
||||||
|
|
||||||
writeColumn(
|
writeColumn(
|
||||||
|
@ -1267,7 +1318,7 @@ public class IndexMaker
|
||||||
CompressedLongsIndexedSupplier compressedLongs = CompressedLongsIndexedSupplier.fromLongBuffer(
|
CompressedLongsIndexedSupplier compressedLongs = CompressedLongsIndexedSupplier.fromLongBuffer(
|
||||||
LongBuffer.wrap(arr),
|
LongBuffer.wrap(arr),
|
||||||
IndexIO.BYTE_ORDER,
|
IndexIO.BYTE_ORDER,
|
||||||
CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
|
compressionStrategy
|
||||||
);
|
);
|
||||||
|
|
||||||
writeColumn(
|
writeColumn(
|
||||||
|
@ -1324,7 +1375,8 @@ public class IndexMaker
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final Set<String> skippedDimensions,
|
final Set<String> skippedDimensions,
|
||||||
final ProgressIndicator progress
|
final ProgressIndicator progress,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final String section = "building index.drd";
|
final String section = "building index.drd";
|
||||||
|
@ -1350,7 +1402,7 @@ 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 String bitmapSerdeFactoryType = mapper.writeValueAsString(bitmapSerdeFactory);
|
final String bitmapSerdeFactoryType = mapper.writeValueAsString(indexSpec.getBitmapSerdeFactory());
|
||||||
final long numBytes = cols.getSerializedSize()
|
final long numBytes = cols.getSerializedSize()
|
||||||
+ dims.getSerializedSize()
|
+ dims.getSerializedSize()
|
||||||
+ 16
|
+ 16
|
||||||
|
|
|
@ -109,7 +109,6 @@ public class IndexMerger
|
||||||
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.makeStartupInjectorWithModules(
|
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
|
||||||
|
@ -125,13 +124,12 @@ public class IndexMerger
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
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, IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
return persist(index, index.getInterval(), outDir);
|
return persist(index, index.getInterval(), outDir, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -146,13 +144,13 @@ public class IndexMerger
|
||||||
*
|
*
|
||||||
* @throws java.io.IOException if an IO error occurs persisting the index
|
* @throws java.io.IOException if an IO error occurs persisting the index
|
||||||
*/
|
*/
|
||||||
public static File persist(final IncrementalIndex index, final Interval dataInterval, File outDir) throws IOException
|
public static File persist(final IncrementalIndex index, final Interval dataInterval, File outDir, IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
return persist(index, dataInterval, outDir, new BaseProgressIndicator());
|
return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File persist(
|
public static File persist(
|
||||||
final IncrementalIndex index, final Interval dataInterval, File outDir, ProgressIndicator progress
|
final IncrementalIndex index, final Interval dataInterval, File outDir, IndexSpec indexSpec, ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
if (index.isEmpty()) {
|
if (index.isEmpty()) {
|
||||||
|
@ -183,24 +181,25 @@ public class IndexMerger
|
||||||
new IncrementalIndexAdapter(
|
new IncrementalIndexAdapter(
|
||||||
dataInterval,
|
dataInterval,
|
||||||
index,
|
index,
|
||||||
bitmapSerdeFactory.getBitmapFactory()
|
indexSpec.getBitmapSerdeFactory().getBitmapFactory()
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
index.getMetricAggs(),
|
index.getMetricAggs(),
|
||||||
outDir,
|
outDir,
|
||||||
|
indexSpec,
|
||||||
progress
|
progress
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File mergeQueryableIndex(
|
public static File mergeQueryableIndex(
|
||||||
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir
|
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return mergeQueryableIndex(indexes, metricAggs, outDir, new BaseProgressIndicator());
|
return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File mergeQueryableIndex(
|
public static File mergeQueryableIndex(
|
||||||
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, ProgressIndicator progress
|
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return merge(
|
return merge(
|
||||||
|
@ -217,19 +216,20 @@ public class IndexMerger
|
||||||
),
|
),
|
||||||
metricAggs,
|
metricAggs,
|
||||||
outDir,
|
outDir,
|
||||||
|
indexSpec,
|
||||||
progress
|
progress
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File merge(
|
public static File merge(
|
||||||
List<IndexableAdapter> indexes, final AggregatorFactory[] metricAggs, File outDir
|
List<IndexableAdapter> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return merge(indexes, metricAggs, outDir, new BaseProgressIndicator());
|
return merge(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File merge(
|
public static File merge(
|
||||||
List<IndexableAdapter> indexes, final AggregatorFactory[] metricAggs, File outDir, ProgressIndicator progress
|
List<IndexableAdapter> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
FileUtils.deleteDirectory(outDir);
|
FileUtils.deleteDirectory(outDir);
|
||||||
|
@ -316,18 +316,18 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn);
|
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File append(
|
public static File append(
|
||||||
List<IndexableAdapter> indexes, File outDir
|
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
return append(indexes, outDir, new BaseProgressIndicator());
|
return append(indexes, outDir, indexSpec, new BaseProgressIndicator());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static File append(
|
public static File append(
|
||||||
List<IndexableAdapter> indexes, File outDir, ProgressIndicator progress
|
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
FileUtils.deleteDirectory(outDir);
|
FileUtils.deleteDirectory(outDir);
|
||||||
|
@ -396,7 +396,7 @@ public class IndexMerger
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn);
|
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, rowMergerFn, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static File makeIndexFiles(
|
private static File makeIndexFiles(
|
||||||
|
@ -405,7 +405,8 @@ public class IndexMerger
|
||||||
final ProgressIndicator progress,
|
final ProgressIndicator progress,
|
||||||
final List<String> mergedDimensions,
|
final List<String> mergedDimensions,
|
||||||
final List<String> mergedMetrics,
|
final List<String> mergedMetrics,
|
||||||
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn
|
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn,
|
||||||
|
final IndexSpec indexSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||||
|
@ -465,7 +466,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));
|
serializerUtils.writeString(channel, mapper.writeValueAsString(indexSpec.getBitmapSerdeFactory()));
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
CloseQuietly.close(channel);
|
CloseQuietly.close(channel);
|
||||||
|
@ -764,6 +765,7 @@ 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());
|
||||||
|
|
||||||
|
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
||||||
GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
|
GenericIndexedWriter<ImmutableBitmap> writer = new GenericIndexedWriter<>(
|
||||||
ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy()
|
ioPeon, dimension, bitmapSerdeFactory.getObjectStrategy()
|
||||||
);
|
);
|
||||||
|
@ -875,10 +877,11 @@ public class IndexMerger
|
||||||
v8OutDir,
|
v8OutDir,
|
||||||
GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy),
|
GenericIndexed.fromIterable(mergedDimensions, GenericIndexed.stringStrategy),
|
||||||
GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy),
|
GenericIndexed.fromIterable(mergedMetrics, GenericIndexed.stringStrategy),
|
||||||
dataInterval
|
dataInterval,
|
||||||
|
indexSpec.getBitmapSerdeFactory()
|
||||||
);
|
);
|
||||||
|
|
||||||
IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir);
|
IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir, indexSpec);
|
||||||
FileUtils.deleteDirectory(v8OutDir);
|
FileUtils.deleteDirectory(v8OutDir);
|
||||||
|
|
||||||
return outDir;
|
return outDir;
|
||||||
|
@ -902,7 +905,8 @@ public class IndexMerger
|
||||||
File inDir,
|
File inDir,
|
||||||
GenericIndexed<String> availableDimensions,
|
GenericIndexed<String> availableDimensions,
|
||||||
GenericIndexed<String> availableMetrics,
|
GenericIndexed<String> availableMetrics,
|
||||||
Interval dataInterval
|
Interval dataInterval,
|
||||||
|
BitmapSerdeFactory bitmapSerdeFactory
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
File indexFile = new File(inDir, "index.drd");
|
File indexFile = new File(inDir, "index.drd");
|
||||||
|
|
|
@ -0,0 +1,180 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
|
import io.druid.segment.data.CompressedObjectStrategy;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* IndexSpec defines segment storage format options to be used at indexing time,
|
||||||
|
* such as bitmap type, and column compression formats.
|
||||||
|
*
|
||||||
|
* IndexSpec is specified as part of the TuningConfig for the corresponding index task.
|
||||||
|
*/
|
||||||
|
public class IndexSpec
|
||||||
|
{
|
||||||
|
public static final String UNCOMPRESSED = "uncompressed";
|
||||||
|
public static final String DEFAULT_METRIC_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY.name().toLowerCase();
|
||||||
|
public static final String DEFAULT_DIMENSION_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY.name().toLowerCase();
|
||||||
|
|
||||||
|
private static final Set<String> COMPRESSION_NAMES = Sets.newHashSet(
|
||||||
|
Iterables.transform(
|
||||||
|
Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()),
|
||||||
|
new Function<CompressedObjectStrategy.CompressionStrategy, String>()
|
||||||
|
{
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public String apply(CompressedObjectStrategy.CompressionStrategy input)
|
||||||
|
{
|
||||||
|
return input.name().toLowerCase();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
private final BitmapSerdeFactory bitmapSerdeFactory;
|
||||||
|
private final String dimensionCompression;
|
||||||
|
private final String metricCompression;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an IndexSpec with default parameters
|
||||||
|
*/
|
||||||
|
public IndexSpec()
|
||||||
|
{
|
||||||
|
this(null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an IndexSpec with the given storage format settings.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default.
|
||||||
|
* Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type"
|
||||||
|
* setting, or, if none was set, uses the default @{link BitmapSerde.DefaultBitmapSerdeFactory}
|
||||||
|
*
|
||||||
|
* @param dimensionCompression compression format for dimension columns. The default, null, means no compression
|
||||||
|
*
|
||||||
|
* @param metricCompression compression format for metric columns, null to use the default.
|
||||||
|
* Defaults to @{link CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY}
|
||||||
|
*/
|
||||||
|
@JsonCreator
|
||||||
|
public IndexSpec(
|
||||||
|
@JsonProperty("bitmap") BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
|
@JsonProperty("dimensionCompression") String dimensionCompression,
|
||||||
|
@JsonProperty("metricCompression") String metricCompression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(dimensionCompression == null || dimensionCompression.equals(UNCOMPRESSED) || COMPRESSION_NAMES.contains(dimensionCompression),
|
||||||
|
"Unknown compression type[%s]", dimensionCompression);
|
||||||
|
|
||||||
|
Preconditions.checkArgument(metricCompression == null || COMPRESSION_NAMES.contains(metricCompression),
|
||||||
|
"Unknown compression type[%s]", metricCompression);
|
||||||
|
|
||||||
|
this.bitmapSerdeFactory = bitmapSerdeFactory != null ? bitmapSerdeFactory : IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY;
|
||||||
|
this.metricCompression = metricCompression;
|
||||||
|
this.dimensionCompression = dimensionCompression;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("bitmap")
|
||||||
|
public BitmapSerdeFactory getBitmapSerdeFactory()
|
||||||
|
{
|
||||||
|
return bitmapSerdeFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("dimensionCompression")
|
||||||
|
public String getDimensionCompression()
|
||||||
|
{
|
||||||
|
return dimensionCompression;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty("metricCompression")
|
||||||
|
public String getMetricCompression()
|
||||||
|
{
|
||||||
|
return metricCompression;
|
||||||
|
}
|
||||||
|
|
||||||
|
public CompressedObjectStrategy.CompressionStrategy getMetricCompressionStrategy()
|
||||||
|
{
|
||||||
|
return CompressedObjectStrategy.CompressionStrategy.valueOf(
|
||||||
|
(metricCompression == null ? DEFAULT_METRIC_COMPRESSION : metricCompression).toUpperCase()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public CompressedObjectStrategy.CompressionStrategy getDimensionCompressionStrategy()
|
||||||
|
{
|
||||||
|
return dimensionCompression == null ?
|
||||||
|
dimensionCompressionStrategyForName(DEFAULT_DIMENSION_COMPRESSION) :
|
||||||
|
dimensionCompressionStrategyForName(dimensionCompression);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static CompressedObjectStrategy.CompressionStrategy dimensionCompressionStrategyForName(String compression)
|
||||||
|
{
|
||||||
|
return compression.equals(UNCOMPRESSED) ? null :
|
||||||
|
CompressedObjectStrategy.CompressionStrategy.valueOf(compression.toUpperCase());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexSpec indexSpec = (IndexSpec) o;
|
||||||
|
|
||||||
|
if (bitmapSerdeFactory != null
|
||||||
|
? !bitmapSerdeFactory.equals(indexSpec.bitmapSerdeFactory)
|
||||||
|
: indexSpec.bitmapSerdeFactory != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (dimensionCompression != null
|
||||||
|
? !dimensionCompression.equals(indexSpec.dimensionCompression)
|
||||||
|
: indexSpec.dimensionCompression != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return !(metricCompression != null
|
||||||
|
? !metricCompression.equals(indexSpec.metricCompression)
|
||||||
|
: indexSpec.metricCompression != null);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = bitmapSerdeFactory != null ? bitmapSerdeFactory.hashCode() : 0;
|
||||||
|
result = 31 * result + (dimensionCompression != null ? dimensionCompression.hashCode() : 0);
|
||||||
|
result = 31 * result + (metricCompression != null ? metricCompression.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Iterators;
|
import com.google.common.collect.Iterators;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
public class NullDimensionSelector implements DimensionSelector
|
public class NullDimensionSelector implements DimensionSelector
|
||||||
|
@ -43,6 +44,18 @@ public class NullDimensionSelector implements DimensionSelector
|
||||||
public Iterator<Integer> iterator() {
|
public Iterator<Integer> iterator() {
|
||||||
return Iterators.singletonIterator(0);
|
return Iterators.singletonIterator(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("NullDimensionSelector does not support fill");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -215,6 +215,11 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
||||||
CloseQuietly.close((Closeable) metric);
|
CloseQuietly.close((Closeable) metric);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (Object dimension : dimensions.values()) {
|
||||||
|
if(dimension instanceof Closeable) {
|
||||||
|
CloseQuietly.close((Closeable) dimension);
|
||||||
|
}
|
||||||
|
}
|
||||||
done = true;
|
done = true;
|
||||||
}
|
}
|
||||||
return hasNext;
|
return hasNext;
|
||||||
|
|
|
@ -356,6 +356,18 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset()));
|
return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Maps;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -109,6 +110,18 @@ public class SingleScanTimeDimSelector implements DimensionSelector
|
||||||
{
|
{
|
||||||
return Iterators.singletonIterator(dimensionValueIndex);
|
return Iterators.singletonIterator(dimensionValueIndex);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,24 +17,25 @@
|
||||||
|
|
||||||
package io.druid.segment.column;
|
package io.druid.segment.column;
|
||||||
|
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
import io.druid.segment.data.CachingIndexed;
|
import io.druid.segment.data.CachingIndexed;
|
||||||
import io.druid.segment.data.IndexedInts;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.VSizeIndexed;
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
import io.druid.segment.data.VSizeIndexedInts;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
|
public class SimpleDictionaryEncodedColumn
|
||||||
|
implements DictionaryEncodedColumn
|
||||||
{
|
{
|
||||||
private final VSizeIndexedInts column;
|
private final IndexedInts column;
|
||||||
private final VSizeIndexed multiValueColumn;
|
private final IndexedMultivalue<IndexedInts> multiValueColumn;
|
||||||
private final CachingIndexed<String> cachedLookups;
|
private final CachingIndexed<String> cachedLookups;
|
||||||
|
|
||||||
public SimpleDictionaryEncodedColumn(
|
public SimpleDictionaryEncodedColumn(
|
||||||
VSizeIndexedInts singleValueColumn,
|
IndexedInts singleValueColumn,
|
||||||
VSizeIndexed multiValueColumn,
|
IndexedMultivalue<IndexedInts> multiValueColumn,
|
||||||
CachingIndexed<String> cachedLookups
|
CachingIndexed<String> cachedLookups
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -88,6 +89,13 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
cachedLookups.close();
|
CloseQuietly.close(cachedLookups);
|
||||||
|
|
||||||
|
if(column != null) {
|
||||||
|
column.close();
|
||||||
|
}
|
||||||
|
if(multiValueColumn != null) {
|
||||||
|
multiValueColumn.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -44,4 +45,16 @@ public class ArrayBasedIndexedInts implements IndexedInts
|
||||||
{
|
{
|
||||||
return new IndexedIntsIterator(this);
|
return new IndexedIntsIterator(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -104,4 +105,16 @@ public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<Immu
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,71 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.collect.Ordering;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
|
||||||
|
public class CompressedByteBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<ByteBuffer>
|
||||||
|
{
|
||||||
|
public static final Ordering<Comparable> ORDERING = Ordering.natural().nullsFirst();
|
||||||
|
|
||||||
|
public static CompressedByteBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
|
||||||
|
{
|
||||||
|
return new CompressedByteBufferObjectStrategy(order, compression, sizePer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public CompressedByteBufferObjectStrategy(
|
||||||
|
ByteOrder order,
|
||||||
|
CompressionStrategy compression,
|
||||||
|
final int sizePer
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(
|
||||||
|
order, new BufferConverter<ByteBuffer>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public ByteBuffer convert(ByteBuffer buf)
|
||||||
|
{
|
||||||
|
return buf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(ByteBuffer lhs, ByteBuffer rhs)
|
||||||
|
{
|
||||||
|
return ORDERING.compare(lhs, rhs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int sizeOf(int count)
|
||||||
|
{
|
||||||
|
return count; // 1 byte per element
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ByteBuffer combine(ByteBuffer into, ByteBuffer from)
|
||||||
|
{
|
||||||
|
return into.put(from);
|
||||||
|
}
|
||||||
|
}, compression, sizePer
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.collect.Ordering;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.IntBuffer;
|
||||||
|
|
||||||
|
public class CompressedIntBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<IntBuffer>
|
||||||
|
{
|
||||||
|
public static final Ordering<Comparable> ORDERING = Ordering.natural().nullsFirst();
|
||||||
|
|
||||||
|
public static CompressedIntBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
|
||||||
|
{
|
||||||
|
return new CompressedIntBufferObjectStrategy(order, compression, sizePer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private CompressedIntBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
|
||||||
|
{
|
||||||
|
super(
|
||||||
|
order,
|
||||||
|
new BufferConverter<IntBuffer>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public IntBuffer convert(ByteBuffer buf)
|
||||||
|
{
|
||||||
|
return buf.asIntBuffer();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(IntBuffer lhs, IntBuffer rhs)
|
||||||
|
{
|
||||||
|
return ORDERING.compare(lhs, rhs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int sizeOf(int count)
|
||||||
|
{
|
||||||
|
return count * Ints.BYTES;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IntBuffer combine(ByteBuffer into, IntBuffer from)
|
||||||
|
{
|
||||||
|
return into.asIntBuffer().put(from);
|
||||||
|
}
|
||||||
|
},
|
||||||
|
compression,
|
||||||
|
sizePer
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,357 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.io.Closeables;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
import com.metamx.common.IAE;
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
|
import io.druid.collections.ResourceHolder;
|
||||||
|
import io.druid.collections.StupidResourceHolder;
|
||||||
|
import io.druid.segment.CompressedPools;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.IntBuffer;
|
||||||
|
import java.nio.channels.WritableByteChannel;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class CompressedIntsIndexedSupplier implements WritableSupplier<IndexedInts>
|
||||||
|
{
|
||||||
|
public static final byte version = 0x2;
|
||||||
|
public static final int MAX_INTS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Ints.BYTES;
|
||||||
|
|
||||||
|
|
||||||
|
private final int totalSize;
|
||||||
|
private final int sizePer;
|
||||||
|
private final GenericIndexed<ResourceHolder<IntBuffer>> baseIntBuffers;
|
||||||
|
private final CompressedObjectStrategy.CompressionStrategy compression;
|
||||||
|
|
||||||
|
CompressedIntsIndexedSupplier(
|
||||||
|
int totalSize,
|
||||||
|
int sizePer,
|
||||||
|
GenericIndexed<ResourceHolder<IntBuffer>> baseIntBuffers,
|
||||||
|
CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.totalSize = totalSize;
|
||||||
|
this.sizePer = sizePer;
|
||||||
|
this.baseIntBuffers = baseIntBuffers;
|
||||||
|
this.compression = compression;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts get()
|
||||||
|
{
|
||||||
|
final int div = Integer.numberOfTrailingZeros(sizePer);
|
||||||
|
final int rem = sizePer - 1;
|
||||||
|
final boolean powerOf2 = sizePer == (1 << div);
|
||||||
|
if(powerOf2) {
|
||||||
|
return new CompressedIndexedInts() {
|
||||||
|
@Override
|
||||||
|
public int get(int index)
|
||||||
|
{
|
||||||
|
// optimize division and remainder for powers of 2
|
||||||
|
final int bufferNum = index >> div;
|
||||||
|
|
||||||
|
if (bufferNum != currIndex) {
|
||||||
|
loadBuffer(bufferNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
final int bufferIndex = index & rem;
|
||||||
|
return buffer.get(buffer.position() + bufferIndex);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
} else {
|
||||||
|
return new CompressedIndexedInts();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getSerializedSize()
|
||||||
|
{
|
||||||
|
return 1 + // version
|
||||||
|
4 + // totalSize
|
||||||
|
4 + // sizePer
|
||||||
|
1 + // compressionId
|
||||||
|
baseIntBuffers.getSerializedSize(); // data
|
||||||
|
}
|
||||||
|
|
||||||
|
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||||
|
{
|
||||||
|
channel.write(ByteBuffer.wrap(new byte[]{version}));
|
||||||
|
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
|
||||||
|
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
|
||||||
|
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
|
||||||
|
baseIntBuffers.writeToChannel(channel);
|
||||||
|
}
|
||||||
|
|
||||||
|
public CompressedIntsIndexedSupplier convertByteOrder(ByteOrder order)
|
||||||
|
{
|
||||||
|
return new CompressedIntsIndexedSupplier(
|
||||||
|
totalSize,
|
||||||
|
sizePer,
|
||||||
|
GenericIndexed.fromIterable(baseIntBuffers, CompressedIntBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For testing. Do not use unless you like things breaking
|
||||||
|
*/
|
||||||
|
GenericIndexed<ResourceHolder<IntBuffer>> getBaseIntBuffers()
|
||||||
|
{
|
||||||
|
return baseIntBuffers;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedIntsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order)
|
||||||
|
{
|
||||||
|
byte versionFromBuffer = buffer.get();
|
||||||
|
|
||||||
|
if (versionFromBuffer == version) {
|
||||||
|
final int totalSize = buffer.getInt();
|
||||||
|
final int sizePer = buffer.getInt();
|
||||||
|
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get());
|
||||||
|
return new CompressedIntsIndexedSupplier(
|
||||||
|
totalSize,
|
||||||
|
sizePer,
|
||||||
|
GenericIndexed.read(buffer, CompressedIntBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
throw new IAE("Unknown version[%s]", versionFromBuffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedIntsIndexedSupplier fromIntBuffer(IntBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression)
|
||||||
|
{
|
||||||
|
return fromIntBuffer(buffer, MAX_INTS_IN_BUFFER, byteOrder, compression);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedIntsIndexedSupplier fromIntBuffer(
|
||||||
|
final IntBuffer buffer, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
chunkFactor <= MAX_INTS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor
|
||||||
|
);
|
||||||
|
|
||||||
|
return new CompressedIntsIndexedSupplier(
|
||||||
|
buffer.remaining(),
|
||||||
|
chunkFactor,
|
||||||
|
GenericIndexed.fromIterable(
|
||||||
|
new Iterable<ResourceHolder<IntBuffer>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<ResourceHolder<IntBuffer>> iterator()
|
||||||
|
{
|
||||||
|
return new Iterator<ResourceHolder<IntBuffer>>()
|
||||||
|
{
|
||||||
|
IntBuffer myBuffer = buffer.asReadOnlyBuffer();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return myBuffer.hasRemaining();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ResourceHolder<IntBuffer> next()
|
||||||
|
{
|
||||||
|
IntBuffer retVal = myBuffer.asReadOnlyBuffer();
|
||||||
|
|
||||||
|
if (chunkFactor < myBuffer.remaining()) {
|
||||||
|
retVal.limit(retVal.position() + chunkFactor);
|
||||||
|
}
|
||||||
|
myBuffer.position(myBuffer.position() + retVal.remaining());
|
||||||
|
|
||||||
|
return StupidResourceHolder.create(retVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void remove()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
},
|
||||||
|
CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor)
|
||||||
|
),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedIntsIndexedSupplier fromList(
|
||||||
|
final List<Integer> list , final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
chunkFactor <= MAX_INTS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor
|
||||||
|
);
|
||||||
|
|
||||||
|
return new CompressedIntsIndexedSupplier(
|
||||||
|
list.size(),
|
||||||
|
chunkFactor,
|
||||||
|
GenericIndexed.fromIterable(
|
||||||
|
new Iterable<ResourceHolder<IntBuffer>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<ResourceHolder<IntBuffer>> iterator()
|
||||||
|
{
|
||||||
|
return new Iterator<ResourceHolder<IntBuffer>>()
|
||||||
|
{
|
||||||
|
int position = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return position < list.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ResourceHolder<IntBuffer> next()
|
||||||
|
{
|
||||||
|
IntBuffer retVal = IntBuffer.allocate(chunkFactor);
|
||||||
|
|
||||||
|
if (chunkFactor > list.size() - position) {
|
||||||
|
retVal.limit(list.size() - position);
|
||||||
|
}
|
||||||
|
final List<Integer> ints = list.subList(position, position + retVal.remaining());
|
||||||
|
for(int value : ints) {
|
||||||
|
retVal.put(value);
|
||||||
|
}
|
||||||
|
retVal.rewind();
|
||||||
|
position += retVal.remaining();
|
||||||
|
|
||||||
|
return StupidResourceHolder.create(retVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void remove()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
},
|
||||||
|
CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor)
|
||||||
|
),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CompressedIndexedInts implements IndexedInts
|
||||||
|
{
|
||||||
|
final Indexed<ResourceHolder<IntBuffer>> singleThreadedIntBuffers = baseIntBuffers.singleThreaded();
|
||||||
|
|
||||||
|
int currIndex = -1;
|
||||||
|
ResourceHolder<IntBuffer> holder;
|
||||||
|
IntBuffer buffer;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int get(int index)
|
||||||
|
{
|
||||||
|
final int bufferNum = index / sizePer;
|
||||||
|
final int bufferIndex = index % sizePer;
|
||||||
|
|
||||||
|
if (bufferNum != currIndex) {
|
||||||
|
loadBuffer(bufferNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
return buffer.get(buffer.position() + bufferIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<Integer> iterator()
|
||||||
|
{
|
||||||
|
return new IndexedIntsIterator(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
if (totalSize - index < toFill.length) {
|
||||||
|
throw new IndexOutOfBoundsException(
|
||||||
|
String.format(
|
||||||
|
"Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
int bufferNum = index / sizePer;
|
||||||
|
int bufferIndex = index % sizePer;
|
||||||
|
|
||||||
|
int leftToFill = toFill.length;
|
||||||
|
while (leftToFill > 0) {
|
||||||
|
if (bufferNum != currIndex) {
|
||||||
|
loadBuffer(bufferNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
buffer.mark();
|
||||||
|
buffer.position(buffer.position() + bufferIndex);
|
||||||
|
final int numToGet = Math.min(buffer.remaining(), leftToFill);
|
||||||
|
buffer.get(toFill, toFill.length - leftToFill, numToGet);
|
||||||
|
buffer.reset();
|
||||||
|
leftToFill -= numToGet;
|
||||||
|
++bufferNum;
|
||||||
|
bufferIndex = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void loadBuffer(int bufferNum)
|
||||||
|
{
|
||||||
|
CloseQuietly.close(holder);
|
||||||
|
holder = singleThreadedIntBuffers.get(bufferNum);
|
||||||
|
buffer = holder.get();
|
||||||
|
currIndex = bufferNum;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "CompressedIntsIndexedSupplier_Anonymous{" +
|
||||||
|
"currIndex=" + currIndex +
|
||||||
|
", sizePer=" + sizePer +
|
||||||
|
", numChunks=" + singleThreadedIntBuffers.size() +
|
||||||
|
", totalSize=" + totalSize +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
Closeables.close(holder, false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -34,6 +34,7 @@ import java.nio.ByteOrder;
|
||||||
import java.nio.LongBuffer;
|
import java.nio.LongBuffer;
|
||||||
import java.nio.channels.WritableByteChannel;
|
import java.nio.channels.WritableByteChannel;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -214,6 +215,65 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static CompressedLongsIndexedSupplier fromList(
|
||||||
|
final List<Long> list , final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
chunkFactor <= MAX_LONGS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor
|
||||||
|
);
|
||||||
|
|
||||||
|
return new CompressedLongsIndexedSupplier(
|
||||||
|
list.size(),
|
||||||
|
chunkFactor,
|
||||||
|
GenericIndexed.fromIterable(
|
||||||
|
new Iterable<ResourceHolder<LongBuffer>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<ResourceHolder<LongBuffer>> iterator()
|
||||||
|
{
|
||||||
|
return new Iterator<ResourceHolder<LongBuffer>>()
|
||||||
|
{
|
||||||
|
int position = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return position < list.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ResourceHolder<LongBuffer> next()
|
||||||
|
{
|
||||||
|
LongBuffer retVal = LongBuffer.allocate(chunkFactor);
|
||||||
|
|
||||||
|
if (chunkFactor > list.size() - position) {
|
||||||
|
retVal.limit(list.size() - position);
|
||||||
|
}
|
||||||
|
final List<Long> longs = list.subList(position, position + retVal.remaining());
|
||||||
|
for (long value : longs) {
|
||||||
|
retVal.put(value);
|
||||||
|
}
|
||||||
|
retVal.rewind();
|
||||||
|
position += retVal.remaining();
|
||||||
|
|
||||||
|
return StupidResourceHolder.create(retVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void remove()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
},
|
||||||
|
CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor)
|
||||||
|
),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private class CompressedIndexedLongs implements IndexedLongs
|
private class CompressedIndexedLongs implements IndexedLongs
|
||||||
{
|
{
|
||||||
final Indexed<ResourceHolder<LongBuffer>> singleThreadedLongBuffers = baseLongBuffers.singleThreaded();
|
final Indexed<ResourceHolder<LongBuffer>> singleThreadedLongBuffers = baseLongBuffers.singleThreaded();
|
||||||
|
|
|
@ -0,0 +1,395 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.io.Closeables;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
import com.google.common.primitives.Shorts;
|
||||||
|
import com.metamx.common.IAE;
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
|
import io.druid.collections.ResourceHolder;
|
||||||
|
import io.druid.collections.StupidResourceHolder;
|
||||||
|
import io.druid.segment.CompressedPools;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.IntBuffer;
|
||||||
|
import java.nio.ShortBuffer;
|
||||||
|
import java.nio.channels.WritableByteChannel;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<IndexedInts>
|
||||||
|
{
|
||||||
|
public static final byte version = 0x2;
|
||||||
|
|
||||||
|
private final int totalSize;
|
||||||
|
private final int sizePer;
|
||||||
|
private final int numBytes;
|
||||||
|
private final int bigEndianShift;
|
||||||
|
private final int littleEndianMask;
|
||||||
|
private final GenericIndexed<ResourceHolder<ByteBuffer>> baseBuffers;
|
||||||
|
private final CompressedObjectStrategy.CompressionStrategy compression;
|
||||||
|
|
||||||
|
CompressedVSizeIntsIndexedSupplier(
|
||||||
|
int totalSize,
|
||||||
|
int sizePer,
|
||||||
|
int numBytes,
|
||||||
|
GenericIndexed<ResourceHolder<ByteBuffer>> baseBuffers,
|
||||||
|
CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
sizePer == (1 << Integer.numberOfTrailingZeros(sizePer)),
|
||||||
|
"Number of entries per chunk must be a power of 2"
|
||||||
|
);
|
||||||
|
|
||||||
|
this.totalSize = totalSize;
|
||||||
|
this.sizePer = sizePer;
|
||||||
|
this.baseBuffers = baseBuffers;
|
||||||
|
this.compression = compression;
|
||||||
|
this.numBytes = numBytes;
|
||||||
|
this.bigEndianShift = Integer.SIZE - (numBytes << 3); // numBytes * 8
|
||||||
|
this.littleEndianMask = (int)((1L << (numBytes << 3)) - 1); // set numBytes * 8 lower bits to 1
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int maxIntsInBufferForBytes(int numBytes)
|
||||||
|
{
|
||||||
|
int maxSizePer = (CompressedPools.BUFFER_SIZE - bufferPadding(numBytes)) / numBytes;
|
||||||
|
// round down to the nearest power of 2
|
||||||
|
return 1 << (Integer.SIZE - 1 - Integer.numberOfLeadingZeros(maxSizePer));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int bufferPadding(int numBytes)
|
||||||
|
{
|
||||||
|
// when numBytes == 3 we need to pad the buffer to allow reading an extra byte
|
||||||
|
// beyond the end of the last value, since we use buffer.getInt() to read values.
|
||||||
|
// for numBytes 1, 2 we remove the need for padding by reading bytes or shorts directly.
|
||||||
|
switch (numBytes) {
|
||||||
|
case Shorts.BYTES:
|
||||||
|
case 1:
|
||||||
|
return 0;
|
||||||
|
default:
|
||||||
|
return Ints.BYTES - numBytes;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int maxIntsInBufferForValue(int maxValue)
|
||||||
|
{
|
||||||
|
return maxIntsInBufferForBytes(VSizeIndexedInts.getNumBytesForMax(maxValue));
|
||||||
|
}
|
||||||
|
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts get()
|
||||||
|
{
|
||||||
|
// optimized versions for int, short, and byte columns
|
||||||
|
if(numBytes == Ints.BYTES) {
|
||||||
|
return new CompressedFullSizeIndexedInts();
|
||||||
|
} else if (numBytes == Shorts.BYTES) {
|
||||||
|
return new CompressedShortSizeIndexedInts();
|
||||||
|
} else if (numBytes == 1) {
|
||||||
|
return new CompressedByteSizeIndexedInts();
|
||||||
|
} else {
|
||||||
|
// default version of everything else, i.e. 3-bytes per value
|
||||||
|
return new CompressedVSizeIndexedInts();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public long getSerializedSize()
|
||||||
|
{
|
||||||
|
return 1 + // version
|
||||||
|
1 + // numBytes
|
||||||
|
Ints.BYTES + // totalSize
|
||||||
|
Ints.BYTES + // sizePer
|
||||||
|
1 + // compression id
|
||||||
|
baseBuffers.getSerializedSize(); // data
|
||||||
|
}
|
||||||
|
|
||||||
|
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||||
|
{
|
||||||
|
channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes}));
|
||||||
|
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
|
||||||
|
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
|
||||||
|
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
|
||||||
|
baseBuffers.writeToChannel(channel);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For testing. Do not use unless you like things breaking
|
||||||
|
*/
|
||||||
|
GenericIndexed<ResourceHolder<ByteBuffer>> getBaseBuffers()
|
||||||
|
{
|
||||||
|
return baseBuffers;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedVSizeIntsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order)
|
||||||
|
{
|
||||||
|
byte versionFromBuffer = buffer.get();
|
||||||
|
|
||||||
|
if (versionFromBuffer == version) {
|
||||||
|
final int numBytes = buffer.get();
|
||||||
|
final int totalSize = buffer.getInt();
|
||||||
|
final int sizePer = buffer.getInt();
|
||||||
|
final int chunkBytes = sizePer * numBytes + bufferPadding(numBytes);
|
||||||
|
|
||||||
|
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get());
|
||||||
|
|
||||||
|
return new CompressedVSizeIntsIndexedSupplier(
|
||||||
|
totalSize,
|
||||||
|
sizePer,
|
||||||
|
numBytes,
|
||||||
|
GenericIndexed.read(
|
||||||
|
buffer,
|
||||||
|
CompressedByteBufferObjectStrategy.getBufferForOrder(order, compression, chunkBytes)
|
||||||
|
),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
throw new IAE("Unknown version[%s]", versionFromBuffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static CompressedVSizeIntsIndexedSupplier fromList(
|
||||||
|
final List<Integer> list, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final int numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue);
|
||||||
|
final int chunkBytes = chunkFactor * numBytes + bufferPadding(numBytes);
|
||||||
|
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
chunkFactor <= maxIntsInBufferForBytes(numBytes),
|
||||||
|
"Chunks must be <= 64k bytes. chunkFactor was[%s]",
|
||||||
|
chunkFactor
|
||||||
|
);
|
||||||
|
|
||||||
|
return new CompressedVSizeIntsIndexedSupplier(
|
||||||
|
list.size(),
|
||||||
|
chunkFactor,
|
||||||
|
numBytes,
|
||||||
|
GenericIndexed.fromIterable(
|
||||||
|
new Iterable<ResourceHolder<ByteBuffer>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterator<ResourceHolder<ByteBuffer>> iterator()
|
||||||
|
{
|
||||||
|
return new Iterator<ResourceHolder<ByteBuffer>>()
|
||||||
|
{
|
||||||
|
int position = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasNext()
|
||||||
|
{
|
||||||
|
return position < list.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ResourceHolder<ByteBuffer> next()
|
||||||
|
{
|
||||||
|
ByteBuffer retVal = ByteBuffer
|
||||||
|
.allocate(chunkBytes)
|
||||||
|
.order(byteOrder);
|
||||||
|
|
||||||
|
if (chunkFactor > list.size() - position) {
|
||||||
|
retVal.limit((list.size() - position) * numBytes);
|
||||||
|
} else {
|
||||||
|
retVal.limit(chunkFactor * numBytes);
|
||||||
|
}
|
||||||
|
|
||||||
|
final List<Integer> ints = list.subList(position, position + retVal.remaining() / numBytes);
|
||||||
|
final ByteBuffer buf = ByteBuffer
|
||||||
|
.allocate(Ints.BYTES)
|
||||||
|
.order(byteOrder);
|
||||||
|
final boolean bigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN);
|
||||||
|
for (int value : ints) {
|
||||||
|
buf.putInt(0, value);
|
||||||
|
if (bigEndian) {
|
||||||
|
retVal.put(buf.array(), Ints.BYTES - numBytes, numBytes);
|
||||||
|
} else {
|
||||||
|
retVal.put(buf.array(), 0, numBytes);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
retVal.rewind();
|
||||||
|
position += retVal.remaining() / numBytes;
|
||||||
|
|
||||||
|
return StupidResourceHolder.create(retVal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void remove()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
},
|
||||||
|
CompressedByteBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkBytes)
|
||||||
|
),
|
||||||
|
compression
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CompressedFullSizeIndexedInts extends CompressedVSizeIndexedInts {
|
||||||
|
IntBuffer intBuffer;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void loadBuffer(int bufferNum)
|
||||||
|
{
|
||||||
|
super.loadBuffer(bufferNum);
|
||||||
|
intBuffer = buffer.asIntBuffer();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int _get(int index)
|
||||||
|
{
|
||||||
|
return intBuffer.get(intBuffer.position() + index);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CompressedShortSizeIndexedInts extends CompressedVSizeIndexedInts {
|
||||||
|
ShortBuffer shortBuffer;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void loadBuffer(int bufferNum)
|
||||||
|
{
|
||||||
|
super.loadBuffer(bufferNum);
|
||||||
|
shortBuffer = buffer.asShortBuffer();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int _get(int index)
|
||||||
|
{
|
||||||
|
// removes the need for padding
|
||||||
|
return shortBuffer.get(shortBuffer.position() + index) & 0xFFFF;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CompressedByteSizeIndexedInts extends CompressedVSizeIndexedInts {
|
||||||
|
@Override
|
||||||
|
protected int _get(int index)
|
||||||
|
{
|
||||||
|
// removes the need for padding
|
||||||
|
return buffer.get(buffer.position() + index) & 0xFF;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private class CompressedVSizeIndexedInts implements IndexedInts
|
||||||
|
{
|
||||||
|
final Indexed<ResourceHolder<ByteBuffer>> singleThreadedBuffers = baseBuffers.singleThreaded();
|
||||||
|
|
||||||
|
final int div = Integer.numberOfTrailingZeros(sizePer);
|
||||||
|
final int rem = sizePer - 1;
|
||||||
|
|
||||||
|
int currIndex = -1;
|
||||||
|
ResourceHolder<ByteBuffer> holder;
|
||||||
|
ByteBuffer buffer;
|
||||||
|
boolean bigEndian;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return totalSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the value at the given index into the column.
|
||||||
|
*
|
||||||
|
* Assumes the number of entries in each decompression buffers is a power of two.
|
||||||
|
*
|
||||||
|
* @param index index of the value in the column
|
||||||
|
* @return the value at the given index
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public int get(int index)
|
||||||
|
{
|
||||||
|
// assumes the number of entries in each buffer is a power of 2
|
||||||
|
final int bufferNum = index >> div;
|
||||||
|
|
||||||
|
if (bufferNum != currIndex) {
|
||||||
|
loadBuffer(bufferNum);
|
||||||
|
}
|
||||||
|
|
||||||
|
return _get(index & rem);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the value at the given index in the current decompression buffer
|
||||||
|
*
|
||||||
|
* @param index index of the value in the curent buffer
|
||||||
|
* @return the value at the given index
|
||||||
|
*/
|
||||||
|
protected int _get(final int index)
|
||||||
|
{
|
||||||
|
final int pos = buffer.position() + index * numBytes;
|
||||||
|
// example for numBytes = 3
|
||||||
|
// big-endian: 0x000c0b0a stored 0c 0b 0a XX, read 0x0c0b0aXX >>> 8
|
||||||
|
// little-endian: 0x000c0b0a stored 0a 0b 0c XX, read 0xXX0c0b0a & 0x00FFFFFF
|
||||||
|
return bigEndian ?
|
||||||
|
buffer.getInt(pos) >>> bigEndianShift :
|
||||||
|
buffer.getInt(pos) & littleEndianMask;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<Integer> iterator()
|
||||||
|
{
|
||||||
|
return new IndexedIntsIterator(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void loadBuffer(int bufferNum)
|
||||||
|
{
|
||||||
|
CloseQuietly.close(holder);
|
||||||
|
holder = singleThreadedBuffers.get(bufferNum);
|
||||||
|
buffer = holder.get();
|
||||||
|
currIndex = bufferNum;
|
||||||
|
bigEndian = buffer.order().equals(ByteOrder.BIG_ENDIAN);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "CompressedVSizedIntsIndexedSupplier{" +
|
||||||
|
"currIndex=" + currIndex +
|
||||||
|
", sizePer=" + sizePer +
|
||||||
|
", numChunks=" + singleThreadedBuffers.size() +
|
||||||
|
", totalSize=" + totalSize +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
Closeables.close(holder, false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -97,4 +97,22 @@ public class ConciseBitmapSerdeFactory implements BitmapSerdeFactory
|
||||||
return conciseComparator.compare((WrappedImmutableConciseBitmap) o1, (WrappedImmutableConciseBitmap) o2);
|
return conciseComparator.compare((WrappedImmutableConciseBitmap) o1, (WrappedImmutableConciseBitmap) o2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "ConciseBitmapSerdeFactory{}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
return this == o || o instanceof ConciseBitmapSerdeFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package io.druid.segment.data;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -44,4 +45,16 @@ public class EmptyIndexedInts implements IndexedInts
|
||||||
{
|
{
|
||||||
return ImmutableList.<Integer>of().iterator();
|
return ImmutableList.<Integer>of().iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,11 +17,14 @@
|
||||||
|
|
||||||
package io.druid.segment.data;
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a int an index (array or list lookup abstraction without boxing).
|
* Get a int an index (array or list lookup abstraction without boxing).
|
||||||
*/
|
*/
|
||||||
public interface IndexedInts extends Iterable<Integer>
|
public interface IndexedInts extends Iterable<Integer>, Closeable
|
||||||
{
|
{
|
||||||
int size();
|
int size();
|
||||||
int get(int index);
|
int get(int index);
|
||||||
|
void fill(int index, int[] toFill);
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,26 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
|
||||||
|
public interface IndexedMultivalue<T extends IndexedInts> extends Indexed<T>, Closeable
|
||||||
|
{
|
||||||
|
}
|
|
@ -21,6 +21,7 @@ import com.google.common.collect.Ordering;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import io.druid.collections.IntList;
|
import io.druid.collections.IntList;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.IntBuffer;
|
import java.nio.IntBuffer;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -120,4 +121,16 @@ public class IntBufferIndexedInts implements IndexedInts, Comparable<IntBufferIn
|
||||||
return Ordering.natural().nullsFirst().compare(o1, o2);
|
return Ordering.natural().nullsFirst().compare(o1, o2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -98,4 +98,22 @@ public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory
|
||||||
return roaringComparator.compare((WrappedImmutableRoaringBitmap) o1, (WrappedImmutableRoaringBitmap) o2);
|
return roaringComparator.compare((WrappedImmutableRoaringBitmap) o1, (WrappedImmutableRoaringBitmap) o2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "RoaringBitmapSerdeFactory{}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
return this == o || o instanceof RoaringBitmapSerdeFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,7 +29,7 @@ import java.util.Iterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class VSizeIndexed implements Indexed<VSizeIndexedInts>
|
public class VSizeIndexed implements IndexedMultivalue<IndexedInts>
|
||||||
{
|
{
|
||||||
private static final byte version = 0x1;
|
private static final byte version = 0x1;
|
||||||
|
|
||||||
|
@ -140,7 +140,7 @@ public class VSizeIndexed implements Indexed<VSizeIndexedInts>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int indexOf(VSizeIndexedInts value)
|
public int indexOf(IndexedInts value)
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException("Reverse lookup not allowed.");
|
throw new UnsupportedOperationException("Reverse lookup not allowed.");
|
||||||
}
|
}
|
||||||
|
@ -176,8 +176,44 @@ public class VSizeIndexed implements Indexed<VSizeIndexedInts>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<VSizeIndexedInts> iterator()
|
public Iterator<IndexedInts> iterator()
|
||||||
{
|
{
|
||||||
return IndexedIterable.create(this).iterator();
|
return IndexedIterable.create(this).iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
// no-op
|
||||||
|
}
|
||||||
|
|
||||||
|
public WritableSupplier<IndexedMultivalue<IndexedInts>> asWritableSupplier() {
|
||||||
|
return new VSizeIndexedSupplier(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class VSizeIndexedSupplier implements WritableSupplier<IndexedMultivalue<IndexedInts>> {
|
||||||
|
final VSizeIndexed delegate;
|
||||||
|
|
||||||
|
public VSizeIndexedSupplier(VSizeIndexed delegate) {
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSerializedSize()
|
||||||
|
{
|
||||||
|
return delegate.getSerializedSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||||
|
{
|
||||||
|
delegate.writeToChannel(channel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedMultivalue<IndexedInts> get()
|
||||||
|
{
|
||||||
|
return delegate;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -151,7 +151,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
||||||
return numBytes;
|
return numBytes;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getSerializedSize()
|
public long getSerializedSize()
|
||||||
{
|
{
|
||||||
// version, numBytes, size, remaining
|
// version, numBytes, size, remaining
|
||||||
return 1 + 1 + 4 + buffer.remaining();
|
return 1 + 1 + 4 + buffer.remaining();
|
||||||
|
@ -190,4 +190,45 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
||||||
throw new IAE("Unknown version[%s]", versionFromBuffer);
|
throw new IAE("Unknown version[%s]", versionFromBuffer);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public WritableSupplier<IndexedInts> asWritableSupplier() {
|
||||||
|
return new VSizeIndexedIntsSupplier(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class VSizeIndexedIntsSupplier implements WritableSupplier<IndexedInts> {
|
||||||
|
final VSizeIndexedInts delegate;
|
||||||
|
|
||||||
|
public VSizeIndexedIntsSupplier(VSizeIndexedInts delegate) {
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getSerializedSize()
|
||||||
|
{
|
||||||
|
return delegate.getSerializedSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||||
|
{
|
||||||
|
delegate.writeToChannel(channel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts get()
|
||||||
|
{
|
||||||
|
return delegate;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.base.Supplier;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.channels.WritableByteChannel;
|
||||||
|
|
||||||
|
public interface WritableSupplier<T> extends Supplier<T>
|
||||||
|
{
|
||||||
|
long getSerializedSize();
|
||||||
|
void writeToChannel(WritableByteChannel channel) throws IOException;
|
||||||
|
}
|
|
@ -53,6 +53,7 @@ import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -198,6 +199,18 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
|
||||||
{
|
{
|
||||||
return vals.iterator();
|
return vals.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.joda.time.Interval;
|
||||||
import org.roaringbitmap.IntIterator;
|
import org.roaringbitmap.IntIterator;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -272,6 +273,18 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -52,6 +52,7 @@ 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.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -328,6 +329,18 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||||
{
|
{
|
||||||
return vals.iterator();
|
return vals.iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,66 +19,230 @@ 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.base.Preconditions;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
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 com.metamx.common.Pair;
|
||||||
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.BitmapSerde;
|
import io.druid.segment.data.BitmapSerde;
|
||||||
import io.druid.segment.data.BitmapSerdeFactory;
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
import io.druid.segment.data.ByteBufferSerializer;
|
import io.druid.segment.data.ByteBufferSerializer;
|
||||||
|
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
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 io.druid.segment.data.WritableSupplier;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import javax.validation.constraints.NotNull;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
import java.nio.channels.WritableByteChannel;
|
import java.nio.channels.WritableByteChannel;
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
{
|
{
|
||||||
private final boolean isSingleValued;
|
private static final int NO_FLAGS = 0;
|
||||||
|
|
||||||
|
enum Feature {
|
||||||
|
MULTI_VALUE;
|
||||||
|
public boolean isSet(int flags) { return (getMask() & flags) != 0; }
|
||||||
|
public int getMask() { return (1 << ordinal()); }
|
||||||
|
}
|
||||||
|
|
||||||
|
enum VERSION
|
||||||
|
{
|
||||||
|
UNCOMPRESSED_SINGLE_VALUE, // 0x0
|
||||||
|
UNCOMPRESSED_MULTI_VALUE, // 0x1
|
||||||
|
COMPRESSED; // 0x2
|
||||||
|
|
||||||
|
public static VERSION fromByte(byte b) {
|
||||||
|
final VERSION[] values = VERSION.values();
|
||||||
|
Preconditions.checkArgument(b < values.length, "Unsupported dictionary column version[%s]", b);
|
||||||
|
return values[b];
|
||||||
|
}
|
||||||
|
|
||||||
|
public byte asByte() {
|
||||||
|
return (byte)this.ordinal();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
private VERSION version = null;
|
||||||
|
private int flags = NO_FLAGS;
|
||||||
|
private GenericIndexed<String> dictionary = null;
|
||||||
|
private WritableSupplier<IndexedInts> singleValuedColumn = null;
|
||||||
|
private WritableSupplier<IndexedMultivalue<IndexedInts>> multiValuedColumn = null;
|
||||||
|
private BitmapSerdeFactory bitmapSerdeFactory = null;
|
||||||
|
private GenericIndexed<ImmutableBitmap> bitmaps = null;
|
||||||
|
private ImmutableRTree spatialIndex = null;
|
||||||
|
private ByteOrder byteOrder = null;
|
||||||
|
|
||||||
|
private Builder()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withDictionary(GenericIndexed<String> dictionary)
|
||||||
|
{
|
||||||
|
this.dictionary = dictionary;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory)
|
||||||
|
{
|
||||||
|
this.bitmapSerdeFactory = bitmapSerdeFactory;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withBitmaps(GenericIndexed<ImmutableBitmap> bitmaps)
|
||||||
|
{
|
||||||
|
this.bitmaps = bitmaps;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withSpatialIndex(ImmutableRTree spatialIndex)
|
||||||
|
{
|
||||||
|
this.spatialIndex = spatialIndex;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withByteOrder(ByteOrder byteOrder)
|
||||||
|
{
|
||||||
|
this.byteOrder = byteOrder;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn)
|
||||||
|
{
|
||||||
|
Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn");
|
||||||
|
this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE;
|
||||||
|
this.singleValuedColumn = singleValuedColumn.asWritableSupplier();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn)
|
||||||
|
{
|
||||||
|
Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn");
|
||||||
|
this.version = VERSION.COMPRESSED;
|
||||||
|
this.singleValuedColumn = singleValuedColumn;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withMultiValuedColumn(VSizeIndexed multiValuedColumn)
|
||||||
|
{
|
||||||
|
Preconditions.checkState(singleValuedColumn == null, "Cannot set both multiValuedColumn and singleValuedColumn");
|
||||||
|
this.version = VERSION.UNCOMPRESSED_MULTI_VALUE;
|
||||||
|
this.flags |= Feature.MULTI_VALUE.getMask();
|
||||||
|
this.multiValuedColumn = multiValuedColumn.asWritableSupplier();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DictionaryEncodedColumnPartSerde build()
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
singleValuedColumn != null ^ multiValuedColumn != null,
|
||||||
|
"Exactly one of singleValCol[%s] or multiValCol[%s] must be set",
|
||||||
|
singleValuedColumn, multiValuedColumn
|
||||||
|
);
|
||||||
|
|
||||||
|
return new DictionaryEncodedColumnPartSerde(
|
||||||
|
version,
|
||||||
|
flags,
|
||||||
|
dictionary,
|
||||||
|
singleValuedColumn,
|
||||||
|
multiValuedColumn,
|
||||||
|
bitmapSerdeFactory,
|
||||||
|
bitmaps,
|
||||||
|
spatialIndex,
|
||||||
|
byteOrder
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Builder builder()
|
||||||
|
{
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
private final BitmapSerdeFactory bitmapSerdeFactory;
|
private final BitmapSerdeFactory bitmapSerdeFactory;
|
||||||
|
private final ByteOrder byteOrder;
|
||||||
|
|
||||||
private final GenericIndexed<String> dictionary;
|
private final GenericIndexed<String> dictionary;
|
||||||
private final VSizeIndexedInts singleValuedColumn;
|
private final WritableSupplier<IndexedInts> singleValuedColumn;
|
||||||
private final VSizeIndexed multiValuedColumn;
|
private final WritableSupplier<IndexedMultivalue<IndexedInts>> multiValuedColumn;
|
||||||
private final GenericIndexed<ImmutableBitmap> bitmaps;
|
private final GenericIndexed<ImmutableBitmap> bitmaps;
|
||||||
private final ImmutableRTree spatialIndex;
|
private final ImmutableRTree spatialIndex;
|
||||||
|
private final int flags;
|
||||||
|
private final VERSION version;
|
||||||
private final long size;
|
private final long size;
|
||||||
|
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
public DictionaryEncodedColumnPartSerde(
|
public DictionaryEncodedColumnPartSerde(
|
||||||
GenericIndexed<String> dictionary,
|
@Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
VSizeIndexedInts singleValCol,
|
@NotNull @JsonProperty("byteOrder") ByteOrder byteOrder
|
||||||
VSizeIndexed multiValCol,
|
|
||||||
BitmapSerdeFactory bitmapSerdeFactory,
|
|
||||||
GenericIndexed<ImmutableBitmap> bitmaps,
|
|
||||||
ImmutableRTree spatialIndex
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.isSingleValued = multiValCol == null;
|
Preconditions.checkArgument(byteOrder != null, "byte order must be specified");
|
||||||
|
|
||||||
|
this.bitmapSerdeFactory = bitmapSerdeFactory == null
|
||||||
|
? new BitmapSerde.LegacyBitmapSerdeFactory()
|
||||||
|
: bitmapSerdeFactory;
|
||||||
|
this.byteOrder = byteOrder;
|
||||||
|
|
||||||
|
// dummy values
|
||||||
|
this.dictionary = null;
|
||||||
|
this.singleValuedColumn = null;
|
||||||
|
this.multiValuedColumn = null;
|
||||||
|
this.bitmaps = null;
|
||||||
|
this.spatialIndex = null;
|
||||||
|
this.size = -1;
|
||||||
|
this.flags = 0;
|
||||||
|
this.version = VERSION.COMPRESSED;
|
||||||
|
}
|
||||||
|
|
||||||
|
private DictionaryEncodedColumnPartSerde(
|
||||||
|
VERSION version,
|
||||||
|
int flags,
|
||||||
|
GenericIndexed<String> dictionary,
|
||||||
|
WritableSupplier<IndexedInts> singleValuedColumn,
|
||||||
|
WritableSupplier<IndexedMultivalue<IndexedInts>> multiValuedColumn,
|
||||||
|
BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
|
GenericIndexed<ImmutableBitmap> bitmaps,
|
||||||
|
ImmutableRTree spatialIndex,
|
||||||
|
ByteOrder byteOrder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(version.compareTo(VERSION.COMPRESSED) <= 0, "Unsupported version[%s]", version);
|
||||||
|
|
||||||
this.bitmapSerdeFactory = bitmapSerdeFactory;
|
this.bitmapSerdeFactory = bitmapSerdeFactory;
|
||||||
|
this.byteOrder = byteOrder;
|
||||||
|
|
||||||
|
this.version = version;
|
||||||
|
this.flags = flags;
|
||||||
|
|
||||||
this.dictionary = dictionary;
|
this.dictionary = dictionary;
|
||||||
this.singleValuedColumn = singleValCol;
|
this.singleValuedColumn = singleValuedColumn;
|
||||||
this.multiValuedColumn = multiValCol;
|
this.multiValuedColumn = multiValuedColumn;
|
||||||
this.bitmaps = bitmaps;
|
this.bitmaps = bitmaps;
|
||||||
this.spatialIndex = spatialIndex;
|
this.spatialIndex = spatialIndex;
|
||||||
|
|
||||||
long size = dictionary.getSerializedSize();
|
long size = dictionary.getSerializedSize();
|
||||||
if (singleValCol != null && multiValCol == null) {
|
|
||||||
size += singleValCol.getSerializedSize();
|
if (Feature.MULTI_VALUE.isSet(flags)) {
|
||||||
} else if (singleValCol == null && multiValCol != null) {
|
size += multiValuedColumn.getSerializedSize();
|
||||||
size += multiValCol.getSerializedSize();
|
|
||||||
} else {
|
} else {
|
||||||
throw new IAE("Either singleValCol[%s] or multiValCol[%s] must be set", singleValCol, multiValCol);
|
size += singleValuedColumn.getSerializedSize();
|
||||||
}
|
}
|
||||||
|
|
||||||
size += bitmaps.getSerializedSize();
|
size += bitmaps.getSerializedSize();
|
||||||
if (spatialIndex != null) {
|
if (spatialIndex != null) {
|
||||||
size += spatialIndex.size() + Ints.BYTES;
|
size += spatialIndex.size() + Ints.BYTES;
|
||||||
|
@ -87,60 +251,38 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
this.size = size;
|
this.size = size;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonCreator
|
|
||||||
public DictionaryEncodedColumnPartSerde(
|
|
||||||
@JsonProperty("isSingleValued") boolean isSingleValued,
|
|
||||||
@JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory
|
|
||||||
)
|
|
||||||
{
|
|
||||||
this.isSingleValued = isSingleValued;
|
|
||||||
this.bitmapSerdeFactory = bitmapSerdeFactory == null
|
|
||||||
? new BitmapSerde.LegacyBitmapSerdeFactory()
|
|
||||||
: bitmapSerdeFactory;
|
|
||||||
|
|
||||||
this.dictionary = null;
|
|
||||||
this.singleValuedColumn = null;
|
|
||||||
this.multiValuedColumn = null;
|
|
||||||
this.bitmaps = null;
|
|
||||||
this.spatialIndex = null;
|
|
||||||
this.size = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
|
||||||
private boolean isSingleValued()
|
|
||||||
{
|
|
||||||
return isSingleValued;
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public BitmapSerdeFactory getBitmapSerdeFactory()
|
public BitmapSerdeFactory getBitmapSerdeFactory()
|
||||||
{
|
{
|
||||||
return bitmapSerdeFactory;
|
return bitmapSerdeFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@JsonProperty
|
||||||
public long numBytes()
|
public ByteOrder getByteOrder()
|
||||||
{
|
{
|
||||||
return 1 + size;
|
return byteOrder;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void write(WritableByteChannel channel) throws IOException
|
public void write(WritableByteChannel channel) throws IOException
|
||||||
{
|
{
|
||||||
channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued ? 0x0 : 0x1)}));
|
channel.write(ByteBuffer.wrap(new byte[]{version.asByte()}));
|
||||||
|
if(version.compareTo(VERSION.COMPRESSED) >= 0) {
|
||||||
|
channel.write(ByteBuffer.wrap(Ints.toByteArray(flags)));
|
||||||
|
}
|
||||||
|
|
||||||
if (dictionary != null) {
|
if (dictionary != null) {
|
||||||
dictionary.writeToChannel(channel);
|
dictionary.writeToChannel(channel);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isSingleValued()) {
|
if (Feature.MULTI_VALUE.isSet(flags)) {
|
||||||
if (singleValuedColumn != null) {
|
|
||||||
singleValuedColumn.writeToChannel(channel);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if (multiValuedColumn != null) {
|
if (multiValuedColumn != null) {
|
||||||
multiValuedColumn.writeToChannel(channel);
|
multiValuedColumn.writeToChannel(channel);
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
if (singleValuedColumn != null) {
|
||||||
|
singleValuedColumn.writeToChannel(channel);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (bitmaps != null) {
|
if (bitmaps != null) {
|
||||||
|
@ -157,67 +299,114 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
public ColumnPartSerde read(
|
||||||
|
ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig
|
||||||
|
)
|
||||||
{
|
{
|
||||||
final boolean isSingleValued = buffer.get() == 0x0;
|
final VERSION rVersion = VERSION.fromByte(buffer.get());
|
||||||
final GenericIndexed<String> dictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy);
|
final int rFlags;
|
||||||
final VSizeIndexedInts singleValuedColumn;
|
|
||||||
final VSizeIndexed multiValuedColumn;
|
|
||||||
|
|
||||||
builder.setType(ValueType.STRING);
|
if(rVersion.compareTo(VERSION.COMPRESSED) >= 0 ) {
|
||||||
|
rFlags = buffer.getInt();
|
||||||
if (isSingleValued) {
|
|
||||||
singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer);
|
|
||||||
multiValuedColumn = null;
|
|
||||||
builder.setHasMultipleValues(false)
|
|
||||||
.setDictionaryEncodedColumn(
|
|
||||||
new DictionaryEncodedColumnSupplier(
|
|
||||||
dictionary,
|
|
||||||
singleValuedColumn,
|
|
||||||
null,
|
|
||||||
columnConfig.columnCacheSizeBytes()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
} else {
|
} else {
|
||||||
singleValuedColumn = null;
|
rFlags = rVersion.equals(VERSION.UNCOMPRESSED_MULTI_VALUE) ?
|
||||||
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
|
Feature.MULTI_VALUE.getMask() :
|
||||||
builder.setHasMultipleValues(true)
|
NO_FLAGS;
|
||||||
.setDictionaryEncodedColumn(
|
|
||||||
new DictionaryEncodedColumnSupplier(
|
|
||||||
dictionary,
|
|
||||||
null,
|
|
||||||
multiValuedColumn,
|
|
||||||
columnConfig.columnCacheSizeBytes()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(
|
final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags);
|
||||||
|
if(rVersion.equals(VERSION.COMPRESSED) && hasMultipleValues) {
|
||||||
|
throw new IAE("Compressed dictionary encoded columns currently do not support multi-value columns");
|
||||||
|
}
|
||||||
|
|
||||||
|
final GenericIndexed<String> rDictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy);
|
||||||
|
builder.setType(ValueType.STRING);
|
||||||
|
|
||||||
|
final WritableSupplier<IndexedInts> rSingleValuedColumn;
|
||||||
|
final WritableSupplier<IndexedMultivalue<IndexedInts>> rMultiValuedColumn;
|
||||||
|
|
||||||
|
if (rVersion.compareTo(VERSION.COMPRESSED) >= 0) {
|
||||||
|
rSingleValuedColumn = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
|
||||||
|
rMultiValuedColumn = null;
|
||||||
|
} else {
|
||||||
|
Pair<WritableSupplier<IndexedInts>, VSizeIndexed> cols = readUncompressed(rVersion, buffer);
|
||||||
|
rSingleValuedColumn = cols.lhs;
|
||||||
|
rMultiValuedColumn = cols.rhs == null ? null : cols.rhs.asWritableSupplier();
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.setHasMultipleValues(hasMultipleValues)
|
||||||
|
.setDictionaryEncodedColumn(
|
||||||
|
new DictionaryEncodedColumnSupplier(
|
||||||
|
rDictionary,
|
||||||
|
rSingleValuedColumn,
|
||||||
|
rMultiValuedColumn,
|
||||||
|
columnConfig.columnCacheSizeBytes()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
GenericIndexed<ImmutableBitmap> rBitmaps = GenericIndexed.read(
|
||||||
buffer, bitmapSerdeFactory.getObjectStrategy()
|
buffer, bitmapSerdeFactory.getObjectStrategy()
|
||||||
);
|
);
|
||||||
builder.setBitmapIndex(
|
builder.setBitmapIndex(
|
||||||
new BitmapIndexColumnPartSupplier(
|
new BitmapIndexColumnPartSupplier(
|
||||||
bitmapSerdeFactory.getBitmapFactory(),
|
bitmapSerdeFactory.getBitmapFactory(),
|
||||||
bitmaps,
|
rBitmaps,
|
||||||
dictionary
|
rDictionary
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
ImmutableRTree spatialIndex = null;
|
ImmutableRTree rSpatialIndex = null;
|
||||||
if (buffer.hasRemaining()) {
|
if (buffer.hasRemaining()) {
|
||||||
spatialIndex = ByteBufferSerializer.read(
|
rSpatialIndex = ByteBufferSerializer.read(
|
||||||
buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory())
|
buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory())
|
||||||
);
|
);
|
||||||
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(spatialIndex));
|
builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex));
|
||||||
}
|
}
|
||||||
|
|
||||||
return new DictionaryEncodedColumnPartSerde(
|
return new DictionaryEncodedColumnPartSerde(
|
||||||
dictionary,
|
rVersion,
|
||||||
singleValuedColumn,
|
rFlags,
|
||||||
multiValuedColumn,
|
rDictionary,
|
||||||
|
rSingleValuedColumn,
|
||||||
|
rMultiValuedColumn,
|
||||||
bitmapSerdeFactory,
|
bitmapSerdeFactory,
|
||||||
bitmaps,
|
rBitmaps,
|
||||||
spatialIndex
|
rSpatialIndex,
|
||||||
|
byteOrder
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static Pair<WritableSupplier<IndexedInts>, VSizeIndexed> readUncompressed(
|
||||||
|
VERSION version,
|
||||||
|
ByteBuffer buffer
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final WritableSupplier<IndexedInts> singleValuedColumn;
|
||||||
|
final VSizeIndexed multiValuedColumn;
|
||||||
|
|
||||||
|
switch (version) {
|
||||||
|
case UNCOMPRESSED_SINGLE_VALUE:
|
||||||
|
singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier();
|
||||||
|
multiValuedColumn = null;
|
||||||
|
break;
|
||||||
|
|
||||||
|
case UNCOMPRESSED_MULTI_VALUE:
|
||||||
|
singleValuedColumn = null;
|
||||||
|
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new IAE("Unsupported version[%s]", version);
|
||||||
|
}
|
||||||
|
|
||||||
|
return Pair.of(singleValuedColumn, multiValuedColumn);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long numBytes()
|
||||||
|
{
|
||||||
|
return 1 + // version
|
||||||
|
(version.compareTo(VERSION.COMPRESSED) >= 0 ? Ints.BYTES : 0) + // flag if version >= compressed
|
||||||
|
size; // size of everything else (dictionary, bitmaps, column, spatialIndex)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,22 +22,22 @@ import io.druid.segment.column.DictionaryEncodedColumn;
|
||||||
import io.druid.segment.column.SimpleDictionaryEncodedColumn;
|
import io.druid.segment.column.SimpleDictionaryEncodedColumn;
|
||||||
import io.druid.segment.data.CachingIndexed;
|
import io.druid.segment.data.CachingIndexed;
|
||||||
import io.druid.segment.data.GenericIndexed;
|
import io.druid.segment.data.GenericIndexed;
|
||||||
import io.druid.segment.data.VSizeIndexed;
|
import io.druid.segment.data.IndexedInts;
|
||||||
import io.druid.segment.data.VSizeIndexedInts;
|
import io.druid.segment.data.IndexedMultivalue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn>
|
public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn>
|
||||||
{
|
{
|
||||||
private final GenericIndexed<String> dictionary;
|
private final GenericIndexed<String> dictionary;
|
||||||
private final VSizeIndexedInts singleValuedColumn;
|
private final Supplier<IndexedInts> singleValuedColumn;
|
||||||
private final VSizeIndexed multiValuedColumn;
|
private final Supplier<IndexedMultivalue<IndexedInts>> multiValuedColumn;
|
||||||
private final int lookupCacheSize;
|
private final int lookupCacheSize;
|
||||||
|
|
||||||
public DictionaryEncodedColumnSupplier(
|
public DictionaryEncodedColumnSupplier(
|
||||||
GenericIndexed<String> dictionary,
|
GenericIndexed<String> dictionary,
|
||||||
VSizeIndexedInts singleValuedColumn,
|
Supplier<IndexedInts> singleValuedColumn,
|
||||||
VSizeIndexed multiValuedColumn,
|
Supplier<IndexedMultivalue<IndexedInts>> multiValuedColumn,
|
||||||
int lookupCacheSize
|
int lookupCacheSize
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -51,8 +51,8 @@ public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncod
|
||||||
public DictionaryEncodedColumn get()
|
public DictionaryEncodedColumn get()
|
||||||
{
|
{
|
||||||
return new SimpleDictionaryEncodedColumn(
|
return new SimpleDictionaryEncodedColumn(
|
||||||
singleValuedColumn,
|
singleValuedColumn != null ? singleValuedColumn.get() : null,
|
||||||
multiValuedColumn,
|
multiValuedColumn != null ? multiValuedColumn.get() : null,
|
||||||
new CachingIndexed<>(dictionary, lookupCacheSize)
|
new CachingIndexed<>(dictionary, lookupCacheSize)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,6 +34,7 @@ import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -125,6 +126,18 @@ public class CardinalityAggregatorTest
|
||||||
{
|
{
|
||||||
return Iterators.forArray(column.get(p));
|
return Iterators.forArray(column.get(p));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(int index, int[] toFill)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("fill not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -60,7 +60,8 @@ public class EmptyIndexTest
|
||||||
IndexMerger.merge(
|
IndexMerger.merge(
|
||||||
Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter),
|
Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter),
|
||||||
new AggregatorFactory[0],
|
new AggregatorFactory[0],
|
||||||
tmpDir
|
tmpDir,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir);
|
QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir);
|
||||||
|
|
|
@ -26,22 +26,61 @@ import io.druid.granularity.QueryGranularity;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
|
import io.druid.segment.data.CompressedObjectStrategy;
|
||||||
|
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||||
import io.druid.segment.data.IncrementalIndexTest;
|
import io.druid.segment.data.IncrementalIndexTest;
|
||||||
|
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||||
import io.druid.segment.incremental.IncrementalIndex;
|
import io.druid.segment.incremental.IncrementalIndex;
|
||||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
/**
|
@RunWith(Parameterized.class)
|
||||||
*/
|
|
||||||
public class IndexMergerTest
|
public class IndexMergerTest
|
||||||
{
|
{
|
||||||
static {
|
@Parameterized.Parameters(name = "{index}: bitmap={0}, compression={1}")
|
||||||
|
public static Collection<Object[]> data()
|
||||||
|
{
|
||||||
|
return Arrays.asList(
|
||||||
|
new Object[][]{
|
||||||
|
{ null, null },
|
||||||
|
{ new RoaringBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZ4 },
|
||||||
|
{ new ConciseBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZ4 },
|
||||||
|
{ new RoaringBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZF},
|
||||||
|
{ new ConciseBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZF},
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
static IndexSpec makeIndexSpec(
|
||||||
|
BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
|
CompressedObjectStrategy.CompressionStrategy compressionStrategy
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if(bitmapSerdeFactory != null || compressionStrategy != null) {
|
||||||
|
return new IndexSpec(
|
||||||
|
bitmapSerdeFactory,
|
||||||
|
compressionStrategy.name().toLowerCase(),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return new IndexSpec();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
|
|
||||||
|
public IndexMergerTest(BitmapSerdeFactory bitmapSerdeFactory, CompressedObjectStrategy.CompressionStrategy compressionStrategy)
|
||||||
|
{
|
||||||
|
this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -54,7 +93,7 @@ public class IndexMergerTest
|
||||||
|
|
||||||
final File tempDir = Files.createTempDir();
|
final File tempDir = Files.createTempDir();
|
||||||
try {
|
try {
|
||||||
QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir));
|
QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir, indexSpec));
|
||||||
|
|
||||||
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()));
|
||||||
|
@ -94,13 +133,13 @@ public class IndexMergerTest
|
||||||
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(IndexMerger.persist(toPersist1, tempDir1));
|
QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1, indexSpec));
|
||||||
|
|
||||||
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(3, index1.getColumnNames().size());
|
Assert.assertEquals(3, index1.getColumnNames().size());
|
||||||
|
|
||||||
QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2));
|
QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2, indexSpec));
|
||||||
|
|
||||||
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()));
|
||||||
|
@ -110,7 +149,8 @@ public class IndexMergerTest
|
||||||
IndexMerger.mergeQueryableIndex(
|
IndexMerger.mergeQueryableIndex(
|
||||||
Arrays.asList(index1, index2),
|
Arrays.asList(index1, index2),
|
||||||
new AggregatorFactory[]{new CountAggregatorFactory("count")},
|
new AggregatorFactory[]{new CountAggregatorFactory("count")},
|
||||||
mergedDir
|
mergedDir,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -151,10 +191,10 @@ public class IndexMergerTest
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1));
|
final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1, indexSpec));
|
||||||
final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2));
|
final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2, indexSpec));
|
||||||
final QueryableIndex merged = IndexIO.loadIndex(
|
final QueryableIndex merged = IndexIO.loadIndex(
|
||||||
IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3)
|
IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3, indexSpec)
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());
|
Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength());
|
||||||
|
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
|
import io.druid.segment.data.CompressedObjectStrategy;
|
||||||
|
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||||
|
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class IndexSpecTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testConfiguredBitmap() throws Exception
|
||||||
|
{
|
||||||
|
// this is just to make sure testSerde correctly tests the bitmap type override
|
||||||
|
Assert.assertEquals(new ConciseBitmapSerdeFactory(), IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws Exception
|
||||||
|
{
|
||||||
|
final ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
final String json = "{ \"bitmap\" : { \"type\" : \"roaring\" }, \"dimensionCompression\" : \"lz4\", \"metricCompression\" : \"lzf\" }";
|
||||||
|
|
||||||
|
final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class);
|
||||||
|
Assert.assertEquals(new RoaringBitmapSerdeFactory(), spec.getBitmapSerdeFactory());
|
||||||
|
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompressionStrategy());
|
||||||
|
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZF, spec.getMetricCompressionStrategy());
|
||||||
|
|
||||||
|
Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerdeUncompressed() throws Exception
|
||||||
|
{
|
||||||
|
final ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
final String json = "{ \"dimensionCompression\" : \"uncompressed\" }";
|
||||||
|
|
||||||
|
final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(IndexSpec.UNCOMPRESSED, spec.getDimensionCompression());
|
||||||
|
Assert.assertEquals(null, spec.getDimensionCompressionStrategy());
|
||||||
|
Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDefaults() throws Exception
|
||||||
|
{
|
||||||
|
final IndexSpec spec = new IndexSpec();
|
||||||
|
Assert.assertEquals(IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY, spec.getBitmapSerdeFactory());
|
||||||
|
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompressionStrategy());
|
||||||
|
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompressionStrategy());
|
||||||
|
}
|
||||||
|
}
|
|
@ -77,6 +77,8 @@ public class SchemalessIndex
|
||||||
new CountAggregatorFactory("count")
|
new CountAggregatorFactory("count")
|
||||||
};
|
};
|
||||||
|
|
||||||
|
private static final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
private static final List<Map<String, Object>> events = Lists.newArrayList();
|
private static final List<Map<String, Object>> events = Lists.newArrayList();
|
||||||
|
|
||||||
private static final Map<Integer, Map<Integer, QueryableIndex>> incrementalIndexes = Maps.newHashMap();
|
private static final Map<Integer, Map<Integer, QueryableIndex>> incrementalIndexes = Maps.newHashMap();
|
||||||
|
@ -184,12 +186,12 @@ public class SchemalessIndex
|
||||||
mergedFile.mkdirs();
|
mergedFile.mkdirs();
|
||||||
mergedFile.deleteOnExit();
|
mergedFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(top, topFile);
|
IndexMerger.persist(top, topFile, indexSpec);
|
||||||
IndexMerger.persist(bottom, bottomFile);
|
IndexMerger.persist(bottom, bottomFile, indexSpec);
|
||||||
|
|
||||||
mergedIndex = io.druid.segment.IndexIO.loadIndex(
|
mergedIndex = io.druid.segment.IndexIO.loadIndex(
|
||||||
IndexMerger.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, indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -231,7 +233,7 @@ public class SchemalessIndex
|
||||||
|
|
||||||
QueryableIndex index = IndexIO.loadIndex(
|
QueryableIndex index = IndexIO.loadIndex(
|
||||||
IndexMerger.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, indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -267,7 +269,7 @@ public class SchemalessIndex
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryableIndex index = IndexIO.loadIndex(
|
QueryableIndex index = IndexIO.loadIndex(
|
||||||
IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile)
|
IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile, indexSpec)
|
||||||
);
|
);
|
||||||
|
|
||||||
return index;
|
return index;
|
||||||
|
@ -348,7 +350,7 @@ public class SchemalessIndex
|
||||||
tmpFile.mkdirs();
|
tmpFile.mkdirs();
|
||||||
tmpFile.deleteOnExit();
|
tmpFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(rowIndex, tmpFile);
|
IndexMerger.persist(rowIndex, tmpFile, indexSpec);
|
||||||
rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile));
|
rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -408,7 +410,7 @@ public class SchemalessIndex
|
||||||
theFile.mkdirs();
|
theFile.mkdirs();
|
||||||
theFile.deleteOnExit();
|
theFile.deleteOnExit();
|
||||||
filesToMap.add(theFile);
|
filesToMap.add(theFile);
|
||||||
IndexMerger.persist(index, theFile);
|
IndexMerger.persist(index, theFile, indexSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
return filesToMap;
|
return filesToMap;
|
||||||
|
@ -482,7 +484,7 @@ public class SchemalessIndex
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile));
|
return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile, indexSpec));
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -521,7 +523,8 @@ public class SchemalessIndex
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
METRIC_AGGS,
|
METRIC_AGGS,
|
||||||
mergedFile
|
mergedFile,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -70,6 +70,7 @@ public class TestIndex
|
||||||
new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]),
|
new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]),
|
||||||
new HyperUniquesAggregatorFactory("quality_uniques", "quality")
|
new HyperUniquesAggregatorFactory("quality_uniques", "quality")
|
||||||
};
|
};
|
||||||
|
private static final IndexSpec indexSpec = new IndexSpec();
|
||||||
|
|
||||||
static {
|
static {
|
||||||
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
|
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
|
||||||
|
@ -131,14 +132,15 @@ public class TestIndex
|
||||||
mergedFile.mkdirs();
|
mergedFile.mkdirs();
|
||||||
mergedFile.deleteOnExit();
|
mergedFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(top, DATA_INTERVAL, topFile);
|
IndexMerger.persist(top, DATA_INTERVAL, topFile, indexSpec);
|
||||||
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile);
|
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile, indexSpec);
|
||||||
|
|
||||||
mergedRealtime = IndexIO.loadIndex(
|
mergedRealtime = IndexIO.loadIndex(
|
||||||
IndexMerger.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,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -243,7 +245,7 @@ public class TestIndex
|
||||||
someTmpFile.mkdirs();
|
someTmpFile.mkdirs();
|
||||||
someTmpFile.deleteOnExit();
|
someTmpFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(index, someTmpFile);
|
IndexMerger.persist(index, someTmpFile, indexSpec);
|
||||||
return IndexIO.loadIndex(someTmpFile);
|
return IndexIO.loadIndex(someTmpFile);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
|
|
|
@ -19,6 +19,7 @@ package io.druid.segment.data;
|
||||||
|
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
import com.google.common.primitives.Floats;
|
import com.google.common.primitives.Floats;
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
import io.druid.segment.CompressedPools;
|
import io.druid.segment.CompressedPools;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -69,6 +70,8 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
|
||||||
private void setupSimple(final int chunkSize)
|
private void setupSimple(final int chunkSize)
|
||||||
{
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
vals = new float[]{
|
vals = new float[]{
|
||||||
0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f
|
0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f
|
||||||
};
|
};
|
||||||
|
@ -94,6 +97,8 @@ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
|
||||||
private void makeWithSerde(int chunkSize) throws IOException
|
private void makeWithSerde(int chunkSize) throws IOException
|
||||||
{
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer(
|
final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer(
|
||||||
FloatBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy
|
FloatBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy
|
||||||
|
|
|
@ -0,0 +1,341 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
import com.google.common.primitives.Longs;
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
|
import io.druid.segment.CompressedPools;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.IntBuffer;
|
||||||
|
import java.nio.channels.Channels;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
{
|
||||||
|
public CompressedIntsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
|
||||||
|
{
|
||||||
|
super(compressionStrategy);
|
||||||
|
}
|
||||||
|
|
||||||
|
private IndexedInts indexed;
|
||||||
|
private CompressedIntsIndexedSupplier supplier;
|
||||||
|
private int[] vals;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
indexed = null;
|
||||||
|
supplier = null;
|
||||||
|
vals = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupSimple(final int chunkSize)
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
|
vals = new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
||||||
|
|
||||||
|
supplier = CompressedIntsIndexedSupplier.fromIntBuffer(
|
||||||
|
IntBuffer.wrap(vals),
|
||||||
|
chunkSize,
|
||||||
|
ByteOrder.nativeOrder(),
|
||||||
|
compressionStrategy
|
||||||
|
);
|
||||||
|
|
||||||
|
indexed = supplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupSimpleWithSerde(final int chunkSize) throws IOException
|
||||||
|
{
|
||||||
|
vals = new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
||||||
|
|
||||||
|
makeWithSerde(chunkSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void makeWithSerde(final int chunkSize) throws IOException
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
|
final CompressedIntsIndexedSupplier theSupplier = CompressedIntsIndexedSupplier.fromIntBuffer(
|
||||||
|
IntBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy
|
||||||
|
);
|
||||||
|
theSupplier.writeToChannel(Channels.newChannel(baos));
|
||||||
|
|
||||||
|
final byte[] bytes = baos.toByteArray();
|
||||||
|
Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length);
|
||||||
|
|
||||||
|
supplier = CompressedIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder());
|
||||||
|
indexed = supplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupLargeChunks(final int chunkSize, final int totalSize) throws IOException
|
||||||
|
{
|
||||||
|
vals = new int[totalSize];
|
||||||
|
Random rand = new Random(0);
|
||||||
|
for(int i = 0; i < vals.length; ++i) {
|
||||||
|
vals[i] = rand.nextInt();
|
||||||
|
}
|
||||||
|
|
||||||
|
makeWithSerde(chunkSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSanity() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(5);
|
||||||
|
|
||||||
|
Assert.assertEquals(4, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
// test powers of 2
|
||||||
|
setupSimple(4);
|
||||||
|
Assert.assertEquals(4, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupSimple(32);
|
||||||
|
Assert.assertEquals(1, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLargeChunks() throws Exception
|
||||||
|
{
|
||||||
|
final int maxChunkSize = CompressedPools.BUFFER_SIZE / Longs.BYTES;
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize, 10 * maxChunkSize);
|
||||||
|
Assert.assertEquals(10, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1);
|
||||||
|
Assert.assertEquals(11, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize - 1, 10 * (maxChunkSize - 1) + 1);
|
||||||
|
Assert.assertEquals(11, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IllegalArgumentException.class)
|
||||||
|
public void testChunkTooBig() throws Exception
|
||||||
|
{
|
||||||
|
final int maxChunkSize = CompressedPools.BUFFER_SIZE / Ints.BYTES;
|
||||||
|
setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBulkFill() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(5);
|
||||||
|
|
||||||
|
tryFill(0, 15);
|
||||||
|
tryFill(3, 6);
|
||||||
|
tryFill(7, 7);
|
||||||
|
tryFill(7, 9);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IndexOutOfBoundsException.class)
|
||||||
|
public void testBulkFillTooMuch() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(5);
|
||||||
|
tryFill(7, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSanityWithSerde() throws Exception
|
||||||
|
{
|
||||||
|
setupSimpleWithSerde(5);
|
||||||
|
|
||||||
|
Assert.assertEquals(4, supplier.getBaseIntBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testBulkFillWithSerde() throws Exception
|
||||||
|
{
|
||||||
|
setupSimpleWithSerde(5);
|
||||||
|
|
||||||
|
tryFill(0, 15);
|
||||||
|
tryFill(3, 6);
|
||||||
|
tryFill(7, 7);
|
||||||
|
tryFill(7, 9);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IndexOutOfBoundsException.class)
|
||||||
|
public void testBulkFillTooMuchWithSerde() throws Exception
|
||||||
|
{
|
||||||
|
setupSimpleWithSerde(5);
|
||||||
|
tryFill(7, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
// This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it,
|
||||||
|
// which sucks but I can't think of a way to deterministically cause it...
|
||||||
|
@Test
|
||||||
|
public void testConcurrentThreadReads() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(5);
|
||||||
|
|
||||||
|
final AtomicReference<String> reason = new AtomicReference<String>("none");
|
||||||
|
|
||||||
|
final int numRuns = 1000;
|
||||||
|
final CountDownLatch startLatch = new CountDownLatch(1);
|
||||||
|
final CountDownLatch stopLatch = new CountDownLatch(2);
|
||||||
|
final AtomicBoolean failureHappened = new AtomicBoolean(false);
|
||||||
|
new Thread(new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set("interrupt.");
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRuns; ++i) {
|
||||||
|
for (int j = 0; j < indexed.size(); ++j) {
|
||||||
|
final long val = vals[j];
|
||||||
|
final long indexedVal = indexed.get(j);
|
||||||
|
if (Longs.compare(val, indexedVal) != 0) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(String.format("Thread1[%d]: %d != %d", j, val, indexedVal));
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(e.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
|
stopLatch.countDown();
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
|
||||||
|
final IndexedInts indexed2 = supplier.get();
|
||||||
|
try {
|
||||||
|
new Thread(new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRuns; ++i) {
|
||||||
|
for (int j = indexed2.size() - 1; j >= 0; --j) {
|
||||||
|
final long val = vals[j];
|
||||||
|
final long indexedVal = indexed2.get(j);
|
||||||
|
if (Longs.compare(val, indexedVal) != 0) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(String.format("Thread2[%d]: %d != %d", j, val, indexedVal));
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
reason.set(e.getMessage());
|
||||||
|
failureHappened.set(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
stopLatch.countDown();
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
|
||||||
|
startLatch.countDown();
|
||||||
|
|
||||||
|
stopLatch.await();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
CloseQuietly.close(indexed2);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (failureHappened.get()) {
|
||||||
|
Assert.fail("Failure happened. Reason: " + reason.get());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void tryFill(final int startIndex, final int size)
|
||||||
|
{
|
||||||
|
int[] filled = new int[size];
|
||||||
|
indexed.fill(startIndex, filled);
|
||||||
|
|
||||||
|
for (int i = startIndex; i < filled.length; i++) {
|
||||||
|
Assert.assertEquals(vals[i + startIndex], filled[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertIndexMatchesVals()
|
||||||
|
{
|
||||||
|
Assert.assertEquals(vals.length, indexed.size());
|
||||||
|
|
||||||
|
// sequential access
|
||||||
|
int[] indices = new int[vals.length];
|
||||||
|
for (int i = 0; i < indexed.size(); ++i) {
|
||||||
|
Assert.assertEquals(vals[i], indexed.get(i), 0.0);
|
||||||
|
indices[i] = i;
|
||||||
|
}
|
||||||
|
|
||||||
|
Collections.shuffle(Arrays.asList(indices));
|
||||||
|
// random access
|
||||||
|
for (int i = 0; i < indexed.size(); ++i) {
|
||||||
|
int k = indices[i];
|
||||||
|
Assert.assertEquals(vals[k], indexed.get(k), 0.0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -69,6 +69,8 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
|
||||||
private void setupSimple(final int chunkSize)
|
private void setupSimple(final int chunkSize)
|
||||||
{
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
||||||
|
|
||||||
supplier = CompressedLongsIndexedSupplier.fromLongBuffer(
|
supplier = CompressedLongsIndexedSupplier.fromLongBuffer(
|
||||||
|
@ -90,6 +92,8 @@ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
|
||||||
private void makeWithSerde(final int chunkSize) throws IOException
|
private void makeWithSerde(final int chunkSize) throws IOException
|
||||||
{
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer(
|
final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer(
|
||||||
LongBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy
|
LongBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy
|
||||||
|
|
|
@ -0,0 +1,371 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.segment.data;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import com.google.common.primitives.Ints;
|
||||||
|
import com.google.common.primitives.Longs;
|
||||||
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
|
import io.druid.segment.CompressedPools;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.channels.Channels;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyTest
|
||||||
|
{
|
||||||
|
@Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}")
|
||||||
|
public static Iterable<Object[]> compressionStrategies()
|
||||||
|
{
|
||||||
|
final Iterable<CompressedObjectStrategy.CompressionStrategy> compressionStrategies = Iterables.transform(
|
||||||
|
CompressionStrategyTest.compressionStrategies(),
|
||||||
|
new Function<Object[], CompressedObjectStrategy.CompressionStrategy>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public CompressedObjectStrategy.CompressionStrategy apply(Object[] input)
|
||||||
|
{
|
||||||
|
return (CompressedObjectStrategy.CompressionStrategy) input[0];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
Set<List<Object>> combinations = Sets.cartesianProduct(
|
||||||
|
Sets.newHashSet(compressionStrategies),
|
||||||
|
Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN)
|
||||||
|
);
|
||||||
|
|
||||||
|
return Iterables.transform(
|
||||||
|
combinations, new Function<List, Object[]>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Object[] apply(List input)
|
||||||
|
{
|
||||||
|
return new Object[]{input.get(0), input.get(1)};
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final int[] MAX_VALUES = new int[] { 0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF };
|
||||||
|
|
||||||
|
public CompressedVSizeIntsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy, ByteOrder byteOrder)
|
||||||
|
{
|
||||||
|
super(compressionStrategy);
|
||||||
|
this.byteOrder = byteOrder;
|
||||||
|
}
|
||||||
|
|
||||||
|
private IndexedInts indexed;
|
||||||
|
private CompressedVSizeIntsIndexedSupplier supplier;
|
||||||
|
private int[] vals;
|
||||||
|
private final ByteOrder byteOrder;
|
||||||
|
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
indexed = null;
|
||||||
|
supplier = null;
|
||||||
|
vals = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupSimple(final int chunkSize)
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
|
vals = new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
||||||
|
|
||||||
|
supplier = CompressedVSizeIntsIndexedSupplier.fromList(
|
||||||
|
Ints.asList(vals),
|
||||||
|
Ints.max(vals),
|
||||||
|
chunkSize,
|
||||||
|
ByteOrder.nativeOrder(),
|
||||||
|
compressionStrategy
|
||||||
|
);
|
||||||
|
|
||||||
|
indexed = supplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupSimpleWithSerde(final int chunkSize) throws IOException
|
||||||
|
{
|
||||||
|
vals = new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16};
|
||||||
|
|
||||||
|
makeWithSerde(chunkSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void makeWithSerde(final int chunkSize) throws IOException
|
||||||
|
{
|
||||||
|
CloseQuietly.close(indexed);
|
||||||
|
|
||||||
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
|
final CompressedVSizeIntsIndexedSupplier theSupplier = CompressedVSizeIntsIndexedSupplier.fromList(
|
||||||
|
Ints.asList(vals), Ints.max(vals), chunkSize, byteOrder, compressionStrategy
|
||||||
|
);
|
||||||
|
theSupplier.writeToChannel(Channels.newChannel(baos));
|
||||||
|
|
||||||
|
final byte[] bytes = baos.toByteArray();
|
||||||
|
Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length);
|
||||||
|
|
||||||
|
supplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), byteOrder);
|
||||||
|
indexed = supplier.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setupLargeChunks(final int chunkSize, final int totalSize, final int maxValue) throws IOException
|
||||||
|
{
|
||||||
|
vals = new int[totalSize];
|
||||||
|
Random rand = new Random(0);
|
||||||
|
for(int i = 0; i < vals.length; ++i) {
|
||||||
|
// VSizeIndexed only allows positive values
|
||||||
|
vals[i] = rand.nextInt(maxValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
makeWithSerde(chunkSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSanity() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(2);
|
||||||
|
Assert.assertEquals(8, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupSimple(4);
|
||||||
|
Assert.assertEquals(4, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupSimple(32);
|
||||||
|
Assert.assertEquals(1, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLargeChunks() throws Exception
|
||||||
|
{
|
||||||
|
for (int maxValue : MAX_VALUES) {
|
||||||
|
final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize, 10 * maxChunkSize, maxValue);
|
||||||
|
Assert.assertEquals(10, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1, maxValue);
|
||||||
|
Assert.assertEquals(11, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupLargeChunks(1, 0xFFFF, maxValue);
|
||||||
|
Assert.assertEquals(0xFFFF, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupLargeChunks(maxChunkSize / 2, 10 * (maxChunkSize / 2) + 1, maxValue);
|
||||||
|
Assert.assertEquals(11, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testChunkTooBig() throws Exception
|
||||||
|
{
|
||||||
|
for(int maxValue : MAX_VALUES) {
|
||||||
|
final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue);
|
||||||
|
try {
|
||||||
|
setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1), maxValue);
|
||||||
|
Assert.fail();
|
||||||
|
} catch(IllegalArgumentException e) {
|
||||||
|
Assert.assertTrue("chunk too big for maxValue " + maxValue, true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testmaxIntsInBuffer() throws Exception
|
||||||
|
{
|
||||||
|
Assert.assertEquals(CompressedPools.BUFFER_SIZE, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(1));
|
||||||
|
Assert.assertEquals(CompressedPools.BUFFER_SIZE / 2, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(2));
|
||||||
|
Assert.assertEquals(CompressedPools.BUFFER_SIZE / 4, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(4));
|
||||||
|
|
||||||
|
Assert.assertEquals(CompressedPools.BUFFER_SIZE, 0x10000); // nearest power of 2 is 2^14
|
||||||
|
Assert.assertEquals(1 << 14, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(3));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSanityWithSerde() throws Exception
|
||||||
|
{
|
||||||
|
setupSimpleWithSerde(4);
|
||||||
|
|
||||||
|
Assert.assertEquals(4, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
|
||||||
|
setupSimpleWithSerde(2);
|
||||||
|
|
||||||
|
Assert.assertEquals(8, supplier.getBaseBuffers().size());
|
||||||
|
assertIndexMatchesVals();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
// This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it,
|
||||||
|
// which sucks but I can't think of a way to deterministically cause it...
|
||||||
|
@Test
|
||||||
|
public void testConcurrentThreadReads() throws Exception
|
||||||
|
{
|
||||||
|
setupSimple(4);
|
||||||
|
|
||||||
|
final AtomicReference<String> reason = new AtomicReference<>("none");
|
||||||
|
|
||||||
|
final int numRuns = 1000;
|
||||||
|
final CountDownLatch startLatch = new CountDownLatch(1);
|
||||||
|
final CountDownLatch stopLatch = new CountDownLatch(2);
|
||||||
|
final AtomicBoolean failureHappened = new AtomicBoolean(false);
|
||||||
|
new Thread(new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set("interrupt.");
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRuns; ++i) {
|
||||||
|
for (int j = 0; j < indexed.size(); ++j) {
|
||||||
|
final long val = vals[j];
|
||||||
|
final long indexedVal = indexed.get(j);
|
||||||
|
if (Longs.compare(val, indexedVal) != 0) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(String.format("Thread1[%d]: %d != %d", j, val, indexedVal));
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(e.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
|
stopLatch.countDown();
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
|
||||||
|
final IndexedInts indexed2 = supplier.get();
|
||||||
|
try {
|
||||||
|
new Thread(new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRuns; ++i) {
|
||||||
|
for (int j = indexed2.size() - 1; j >= 0; --j) {
|
||||||
|
final long val = vals[j];
|
||||||
|
final long indexedVal = indexed2.get(j);
|
||||||
|
if (Longs.compare(val, indexedVal) != 0) {
|
||||||
|
failureHappened.set(true);
|
||||||
|
reason.set(String.format("Thread2[%d]: %d != %d", j, val, indexedVal));
|
||||||
|
stopLatch.countDown();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
reason.set(e.getMessage());
|
||||||
|
failureHappened.set(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
stopLatch.countDown();
|
||||||
|
}
|
||||||
|
}).start();
|
||||||
|
|
||||||
|
startLatch.countDown();
|
||||||
|
|
||||||
|
stopLatch.await();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
CloseQuietly.close(indexed2);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (failureHappened.get()) {
|
||||||
|
Assert.fail("Failure happened. Reason: " + reason.get());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertIndexMatchesVals()
|
||||||
|
{
|
||||||
|
Assert.assertEquals(vals.length, indexed.size());
|
||||||
|
|
||||||
|
// sequential access
|
||||||
|
int[] indices = new int[vals.length];
|
||||||
|
for (int i = 0; i < indexed.size(); ++i) {
|
||||||
|
final int expected = vals[i];
|
||||||
|
final int actual = indexed.get(i);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
indices[i] = i;
|
||||||
|
}
|
||||||
|
|
||||||
|
Collections.shuffle(Arrays.asList(indices));
|
||||||
|
// random access
|
||||||
|
for (int i = 0; i < indexed.size(); ++i) {
|
||||||
|
int k = indices[i];
|
||||||
|
Assert.assertEquals(vals[k], indexed.get(k));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -43,6 +43,7 @@ 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.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
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;
|
||||||
|
@ -86,9 +87,10 @@ public class SpatialFilterBonusTest
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Collection<?> constructorFeeder() throws IOException
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
|
final IndexSpec indexSpec = new IndexSpec();
|
||||||
final IncrementalIndex rtIndex = makeIncrementalIndex();
|
final IncrementalIndex rtIndex = makeIncrementalIndex();
|
||||||
final QueryableIndex mMappedTestIndex = makeQueryableIndex();
|
final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec);
|
||||||
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex();
|
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec);
|
||||||
return Arrays.asList(
|
return Arrays.asList(
|
||||||
new Object[][]{
|
new Object[][]{
|
||||||
{
|
{
|
||||||
|
@ -222,7 +224,7 @@ public class SpatialFilterBonusTest
|
||||||
return theIndex;
|
return theIndex;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static QueryableIndex makeQueryableIndex() throws IOException
|
private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
IncrementalIndex theIndex = makeIncrementalIndex();
|
IncrementalIndex theIndex = makeIncrementalIndex();
|
||||||
File tmpFile = File.createTempFile("billy", "yay");
|
File tmpFile = File.createTempFile("billy", "yay");
|
||||||
|
@ -230,11 +232,11 @@ public class SpatialFilterBonusTest
|
||||||
tmpFile.mkdirs();
|
tmpFile.mkdirs();
|
||||||
tmpFile.deleteOnExit();
|
tmpFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(theIndex, tmpFile);
|
IndexMerger.persist(theIndex, tmpFile, indexSpec);
|
||||||
return IndexIO.loadIndex(tmpFile);
|
return IndexIO.loadIndex(tmpFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static QueryableIndex makeMergedQueryableIndex()
|
private static QueryableIndex makeMergedQueryableIndex(final IndexSpec indexSpec)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
IncrementalIndex first = new OnheapIncrementalIndex(
|
IncrementalIndex first = new OnheapIncrementalIndex(
|
||||||
|
@ -410,15 +412,16 @@ public class SpatialFilterBonusTest
|
||||||
mergedFile.mkdirs();
|
mergedFile.mkdirs();
|
||||||
mergedFile.deleteOnExit();
|
mergedFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(first, DATA_INTERVAL, firstFile);
|
IndexMerger.persist(first, DATA_INTERVAL, firstFile, indexSpec);
|
||||||
IndexMerger.persist(second, DATA_INTERVAL, secondFile);
|
IndexMerger.persist(second, DATA_INTERVAL, secondFile, indexSpec);
|
||||||
IndexMerger.persist(third, DATA_INTERVAL, thirdFile);
|
IndexMerger.persist(third, DATA_INTERVAL, thirdFile, indexSpec);
|
||||||
|
|
||||||
QueryableIndex mergedRealtime = IndexIO.loadIndex(
|
QueryableIndex mergedRealtime = IndexIO.loadIndex(
|
||||||
IndexMerger.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,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -44,6 +44,7 @@ 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.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
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;
|
||||||
|
@ -82,9 +83,10 @@ public class SpatialFilterTest
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Collection<?> constructorFeeder() throws IOException
|
public static Collection<?> constructorFeeder() throws IOException
|
||||||
{
|
{
|
||||||
|
final IndexSpec indexSpec = new IndexSpec();
|
||||||
final IncrementalIndex rtIndex = makeIncrementalIndex();
|
final IncrementalIndex rtIndex = makeIncrementalIndex();
|
||||||
final QueryableIndex mMappedTestIndex = makeQueryableIndex();
|
final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec);
|
||||||
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex();
|
final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec);
|
||||||
return Arrays.asList(
|
return Arrays.asList(
|
||||||
new Object[][]{
|
new Object[][]{
|
||||||
{
|
{
|
||||||
|
@ -251,7 +253,7 @@ public class SpatialFilterTest
|
||||||
return theIndex;
|
return theIndex;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static QueryableIndex makeQueryableIndex() throws IOException
|
private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException
|
||||||
{
|
{
|
||||||
IncrementalIndex theIndex = makeIncrementalIndex();
|
IncrementalIndex theIndex = makeIncrementalIndex();
|
||||||
File tmpFile = File.createTempFile("billy", "yay");
|
File tmpFile = File.createTempFile("billy", "yay");
|
||||||
|
@ -259,11 +261,11 @@ public class SpatialFilterTest
|
||||||
tmpFile.mkdirs();
|
tmpFile.mkdirs();
|
||||||
tmpFile.deleteOnExit();
|
tmpFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(theIndex, tmpFile);
|
IndexMerger.persist(theIndex, tmpFile, indexSpec);
|
||||||
return IndexIO.loadIndex(tmpFile);
|
return IndexIO.loadIndex(tmpFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static QueryableIndex makeMergedQueryableIndex()
|
private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
IncrementalIndex first = new OnheapIncrementalIndex(
|
IncrementalIndex first = new OnheapIncrementalIndex(
|
||||||
|
@ -479,15 +481,16 @@ public class SpatialFilterTest
|
||||||
mergedFile.mkdirs();
|
mergedFile.mkdirs();
|
||||||
mergedFile.deleteOnExit();
|
mergedFile.deleteOnExit();
|
||||||
|
|
||||||
IndexMerger.persist(first, DATA_INTERVAL, firstFile);
|
IndexMerger.persist(first, DATA_INTERVAL, firstFile, indexSpec);
|
||||||
IndexMerger.persist(second, DATA_INTERVAL, secondFile);
|
IndexMerger.persist(second, DATA_INTERVAL, secondFile, indexSpec);
|
||||||
IndexMerger.persist(third, DATA_INTERVAL, thirdFile);
|
IndexMerger.persist(third, DATA_INTERVAL, thirdFile, indexSpec);
|
||||||
|
|
||||||
QueryableIndex mergedRealtime = IndexIO.loadIndex(
|
QueryableIndex mergedRealtime = IndexIO.loadIndex(
|
||||||
IndexMerger.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,
|
||||||
|
indexSpec
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,9 @@ package io.druid.segment.indexing;
|
||||||
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.io.Files;
|
import com.google.common.io.Files;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
|
import io.druid.segment.data.BitmapSerde;
|
||||||
|
import io.druid.segment.data.BitmapSerdeFactory;
|
||||||
import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy;
|
import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy;
|
||||||
import io.druid.segment.realtime.plumber.RejectionPolicyFactory;
|
import io.druid.segment.realtime.plumber.RejectionPolicyFactory;
|
||||||
import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory;
|
import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory;
|
||||||
|
@ -42,6 +45,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
private static final RejectionPolicyFactory defaultRejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
|
private static final RejectionPolicyFactory defaultRejectionPolicyFactory = new ServerTimeRejectionPolicyFactory();
|
||||||
private static final int defaultMaxPendingPersists = 0;
|
private static final int defaultMaxPendingPersists = 0;
|
||||||
private static final ShardSpec defaultShardSpec = new NoneShardSpec();
|
private static final ShardSpec defaultShardSpec = new NoneShardSpec();
|
||||||
|
private static final IndexSpec defaultIndexSpec = new IndexSpec();
|
||||||
private static final boolean defaultPersistInHeap = false;
|
private static final boolean defaultPersistInHeap = false;
|
||||||
private static final boolean defaultIngestOffheap = false;
|
private static final boolean defaultIngestOffheap = false;
|
||||||
private static final int defaultBufferSize = 128 * 1024* 1024; // 128M
|
private static final int defaultBufferSize = 128 * 1024* 1024; // 128M
|
||||||
|
@ -59,6 +63,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
defaultRejectionPolicyFactory,
|
defaultRejectionPolicyFactory,
|
||||||
defaultMaxPendingPersists,
|
defaultMaxPendingPersists,
|
||||||
defaultShardSpec,
|
defaultShardSpec,
|
||||||
|
defaultIndexSpec,
|
||||||
defaultPersistInHeap,
|
defaultPersistInHeap,
|
||||||
defaultIngestOffheap,
|
defaultIngestOffheap,
|
||||||
defaultBufferSize
|
defaultBufferSize
|
||||||
|
@ -73,6 +78,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
private final RejectionPolicyFactory rejectionPolicyFactory;
|
private final RejectionPolicyFactory rejectionPolicyFactory;
|
||||||
private final int maxPendingPersists;
|
private final int maxPendingPersists;
|
||||||
private final ShardSpec shardSpec;
|
private final ShardSpec shardSpec;
|
||||||
|
private final IndexSpec indexSpec;
|
||||||
private final boolean persistInHeap;
|
private final boolean persistInHeap;
|
||||||
private final boolean ingestOffheap;
|
private final boolean ingestOffheap;
|
||||||
private final int bufferSize;
|
private final int bufferSize;
|
||||||
|
@ -87,6 +93,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory,
|
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory,
|
||||||
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
|
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
|
||||||
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
||||||
|
@JsonProperty("indexSpec") IndexSpec indexSpec,
|
||||||
@JsonProperty("persistInHeap") Boolean persistInHeap,
|
@JsonProperty("persistInHeap") Boolean persistInHeap,
|
||||||
@JsonProperty("ingestOffheap") Boolean ingestOffheap,
|
@JsonProperty("ingestOffheap") Boolean ingestOffheap,
|
||||||
@JsonProperty("buffersize") Integer bufferSize
|
@JsonProperty("buffersize") Integer bufferSize
|
||||||
|
@ -104,6 +111,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
: rejectionPolicyFactory;
|
: rejectionPolicyFactory;
|
||||||
this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists;
|
this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists;
|
||||||
this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec;
|
this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec;
|
||||||
|
this.indexSpec = indexSpec == null ? defaultIndexSpec : indexSpec;
|
||||||
this.persistInHeap = persistInHeap == null ? defaultPersistInHeap : persistInHeap;
|
this.persistInHeap = persistInHeap == null ? defaultPersistInHeap : persistInHeap;
|
||||||
this.ingestOffheap = ingestOffheap == null ? defaultIngestOffheap : ingestOffheap;
|
this.ingestOffheap = ingestOffheap == null ? defaultIngestOffheap : ingestOffheap;
|
||||||
this.bufferSize = bufferSize == null ? defaultBufferSize : bufferSize;
|
this.bufferSize = bufferSize == null ? defaultBufferSize : bufferSize;
|
||||||
|
@ -158,6 +166,12 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
return shardSpec;
|
return shardSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public IndexSpec getIndexSpec()
|
||||||
|
{
|
||||||
|
return indexSpec;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public boolean isPersistInHeap()
|
public boolean isPersistInHeap()
|
||||||
{
|
{
|
||||||
|
@ -185,6 +199,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
rejectionPolicyFactory,
|
rejectionPolicyFactory,
|
||||||
maxPendingPersists,
|
maxPendingPersists,
|
||||||
shardSpec,
|
shardSpec,
|
||||||
|
indexSpec,
|
||||||
persistInHeap,
|
persistInHeap,
|
||||||
ingestOffheap,
|
ingestOffheap,
|
||||||
bufferSize
|
bufferSize
|
||||||
|
@ -202,6 +217,7 @@ public class RealtimeTuningConfig implements TuningConfig
|
||||||
rejectionPolicyFactory,
|
rejectionPolicyFactory,
|
||||||
maxPendingPersists,
|
maxPendingPersists,
|
||||||
shardSpec,
|
shardSpec,
|
||||||
|
indexSpec,
|
||||||
persistInHeap,
|
persistInHeap,
|
||||||
ingestOffheap,
|
ingestOffheap,
|
||||||
bufferSize
|
bufferSize
|
||||||
|
|
|
@ -55,6 +55,7 @@ import io.druid.query.spec.SpecificSegmentSpec;
|
||||||
import io.druid.segment.IndexIO;
|
import io.druid.segment.IndexIO;
|
||||||
import io.druid.segment.IndexMaker;
|
import io.druid.segment.IndexMaker;
|
||||||
import io.druid.segment.IndexMerger;
|
import io.druid.segment.IndexMerger;
|
||||||
|
import io.druid.segment.IndexSpec;
|
||||||
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;
|
||||||
|
@ -438,13 +439,15 @@ public class RealtimePlumber implements Plumber
|
||||||
mergedFile = IndexMaker.mergeQueryableIndex(
|
mergedFile = IndexMaker.mergeQueryableIndex(
|
||||||
indexes,
|
indexes,
|
||||||
schema.getAggregators(),
|
schema.getAggregators(),
|
||||||
mergedTarget
|
mergedTarget,
|
||||||
|
config.getIndexSpec()
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
mergedFile = IndexMerger.mergeQueryableIndex(
|
mergedFile = IndexMerger.mergeQueryableIndex(
|
||||||
indexes,
|
indexes,
|
||||||
schema.getAggregators(),
|
schema.getAggregators(),
|
||||||
mergedTarget
|
mergedTarget,
|
||||||
|
config.getIndexSpec()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -833,15 +836,18 @@ public class RealtimePlumber implements Plumber
|
||||||
int numRows = indexToPersist.getIndex().size();
|
int numRows = indexToPersist.getIndex().size();
|
||||||
|
|
||||||
final File persistedFile;
|
final File persistedFile;
|
||||||
|
final IndexSpec indexSpec = config.getIndexSpec();
|
||||||
if (config.isPersistInHeap()) {
|
if (config.isPersistInHeap()) {
|
||||||
persistedFile = IndexMaker.persist(
|
persistedFile = IndexMaker.persist(
|
||||||
indexToPersist.getIndex(),
|
indexToPersist.getIndex(),
|
||||||
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
|
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
persistedFile = IndexMerger.persist(
|
persistedFile = IndexMerger.persist(
|
||||||
indexToPersist.getIndex(),
|
indexToPersist.getIndex(),
|
||||||
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount()))
|
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())),
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -74,7 +74,7 @@ public class FireDepartmentTest
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new RealtimeTuningConfig(
|
new RealtimeTuningConfig(
|
||||||
null, null, null, null, null, null, null, null, false, false, null
|
null, null, null, null, null, null, null, null, null, false, false, null
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -105,6 +105,7 @@ public class RealtimeManagerTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString()));
|
plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString()));
|
||||||
|
|
|
@ -164,6 +164,7 @@ public class RealtimePlumberSchoolTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -62,6 +62,7 @@ public class SinkTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
false,
|
false,
|
||||||
false,
|
false,
|
||||||
null
|
null
|
||||||
|
|
Loading…
Reference in New Issue