Merge pull request #1804 from himanshug/objectify_index_creators

static to non-static conversion for methods in Index[Merger/Maker/IO]
This commit is contained in:
Fangjin Yang 2015-11-03 11:25:32 -08:00
commit cec09a9967
75 changed files with 955 additions and 622 deletions

View File

@ -119,7 +119,7 @@ The indexSpec is optional and default parameters will be used if not specified.
|property|description|possible values|default|required?| |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| |bitmap|type of bitmap compression to use for inverted indices.|`"concise"`, `"roaring"`|`"concise"`|no|
|dimensionCompression|compression format for dimension columns (currently only affects single-value dimensions, multi-value dimensions are always uncompressed)|`"uncompressed"`, `"lz4"`, `"lzf"`|`"lz4"`|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| |metricCompression|compression format for metric columns, defaults to LZ4|`"lz4"`, `"lzf"`|`"lz4"`|no|

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.Reducer;
@ -132,7 +131,7 @@ public class DetermineHashedPartitionsJob implements Jobby
if (!Utils.exists(groupByJob, fileSystem, intervalInfoPath)) { if (!Utils.exists(groupByJob, fileSystem, intervalInfoPath)) {
throw new ISE("Path[%s] didn't exist!?", intervalInfoPath); throw new ISE("Path[%s] didn't exist!?", intervalInfoPath);
} }
List<Interval> intervals = config.jsonMapper.readValue( List<Interval> intervals = config.JSON_MAPPER.readValue(
Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference<List<Interval>>() Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference<List<Interval>>()
{ {
} }
@ -156,7 +155,7 @@ public class DetermineHashedPartitionsJob implements Jobby
fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration()); fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration());
} }
if (Utils.exists(groupByJob, fileSystem, partitionInfoPath)) { if (Utils.exists(groupByJob, fileSystem, partitionInfoPath)) {
final Long numRows = config.jsonMapper.readValue( final Long numRows = config.JSON_MAPPER.readValue(
Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference<Long>() Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference<Long>()
{ {
} }
@ -178,7 +177,7 @@ public class DetermineHashedPartitionsJob implements Jobby
new HashBasedNumberedShardSpec( new HashBasedNumberedShardSpec(
i, i,
numberOfShards, numberOfShards,
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
shardCount++ shardCount++
) )
@ -267,7 +266,7 @@ public class DetermineHashedPartitionsJob implements Jobby
} }
hyperLogLogs.get(interval) hyperLogLogs.get(interval)
.add( .add(
hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)) hashFunction.hashBytes(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey))
.asBytes() .asBytes()
); );
} }
@ -324,7 +323,7 @@ public class DetermineHashedPartitionsJob implements Jobby
); );
try { try {
HadoopDruidIndexerConfig.jsonMapper.writerWithType( HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(
new TypeReference<Long>() new TypeReference<Long>()
{ {
} }
@ -350,7 +349,7 @@ public class DetermineHashedPartitionsJob implements Jobby
); );
try { try {
HadoopDruidIndexerConfig.jsonMapper.writerWithType( HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(
new TypeReference<List<Interval>>() new TypeReference<List<Interval>>()
{ {
} }

View File

@ -49,7 +49,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapred.InvalidJobConfException;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobContext;
@ -92,8 +91,8 @@ public class DeterminePartitionsJob implements Jobby
{ {
private static final Logger log = new Logger(DeterminePartitionsJob.class); private static final Logger log = new Logger(DeterminePartitionsJob.class);
private static final Joiner tabJoiner = HadoopDruidIndexerConfig.tabJoiner; private static final Joiner TAB_JOINER = HadoopDruidIndexerConfig.TAB_JOINER;
private static final Splitter tabSplitter = HadoopDruidIndexerConfig.tabSplitter; private static final Splitter TAB_SPLITTER = HadoopDruidIndexerConfig.TAB_SPLITTER;
private final HadoopDruidIndexerConfig config; private final HadoopDruidIndexerConfig config;
@ -225,7 +224,7 @@ public class DeterminePartitionsJob implements Jobby
fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration()); fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration());
} }
if (Utils.exists(dimSelectionJob, fileSystem, partitionInfoPath)) { if (Utils.exists(dimSelectionJob, fileSystem, partitionInfoPath)) {
List<ShardSpec> specs = config.jsonMapper.readValue( List<ShardSpec> specs = config.JSON_MAPPER.readValue(
Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference<List<ShardSpec>>() Utils.openInputStream(dimSelectionJob, partitionInfoPath), new TypeReference<List<ShardSpec>>()
{ {
} }
@ -275,7 +274,7 @@ public class DeterminePartitionsJob implements Jobby
inputRow inputRow
); );
context.write( context.write(
new BytesWritable(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)), new BytesWritable(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)),
NullWritable.get() NullWritable.get()
); );
} }
@ -317,7 +316,7 @@ public class DeterminePartitionsJob implements Jobby
BytesWritable key, NullWritable value, Context context BytesWritable key, NullWritable value, Context context
) throws IOException, InterruptedException ) throws IOException, InterruptedException
{ {
final List<Object> timeAndDims = HadoopDruidIndexerConfig.jsonMapper.readValue(key.getBytes(), List.class); final List<Object> timeAndDims = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(key.getBytes(), List.class);
final DateTime timestamp = new DateTime(timeAndDims.get(0)); final DateTime timestamp = new DateTime(timeAndDims.get(0));
final Map<String, Iterable<String>> dims = (Map<String, Iterable<String>>) timeAndDims.get(1); final Map<String, Iterable<String>> dims = (Map<String, Iterable<String>>) timeAndDims.get(1);
@ -773,11 +772,11 @@ public class DeterminePartitionsJob implements Jobby
log.info("Chosen partitions:"); log.info("Chosen partitions:");
for (ShardSpec shardSpec : chosenShardSpecs) { for (ShardSpec shardSpec : chosenShardSpecs) {
log.info(" %s", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(shardSpec)); log.info(" %s", HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(shardSpec));
} }
try { try {
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
.writerWithType( .writerWithType(
new TypeReference<List<ShardSpec>>() new TypeReference<List<ShardSpec>>()
{ {
@ -883,12 +882,12 @@ public class DeterminePartitionsJob implements Jobby
public Text toText() public Text toText()
{ {
return new Text(tabJoiner.join(dim, String.valueOf(numRows), value)); return new Text(TAB_JOINER.join(dim, String.valueOf(numRows), value));
} }
public static DimValueCount fromText(Text text) public static DimValueCount fromText(Text text)
{ {
final Iterator<String> splits = tabSplitter.limit(3).split(text.toString()).iterator(); final Iterator<String> splits = TAB_SPLITTER.limit(3).split(text.toString()).iterator();
final String dim = splits.next(); final String dim = splits.next();
final int numRows = Integer.parseInt(splits.next()); final int numRows = Integer.parseInt(splits.next());
final String value = splits.next(); final String value = splits.next();
@ -906,8 +905,8 @@ public class DeterminePartitionsJob implements Jobby
{ {
context.write( context.write(
new SortableBytes( new SortableBytes(
groupKey, tabJoiner.join(dimValueCount.dim, dimValueCount.value).getBytes( groupKey, TAB_JOINER.join(dimValueCount.dim, dimValueCount.value).getBytes(
HadoopDruidIndexerConfig.javaNativeCharset HadoopDruidIndexerConfig.JAVA_NATIVE_CHARSET
) )
).toBytesWritable(), ).toBytesWritable(),
dimValueCount.toText() dimValueCount.toText()

View File

@ -65,7 +65,7 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby
for (int i = 0; i < shardsPerInterval; i++) { for (int i = 0; i < shardsPerInterval; i++) {
specs.add( specs.add(
new HadoopyShardSpec( new HadoopyShardSpec(
new HashBasedNumberedShardSpec(i, shardsPerInterval, HadoopDruidIndexerConfig.jsonMapper), new HashBasedNumberedShardSpec(i, shardsPerInterval, HadoopDruidIndexerConfig.JSON_MAPPER),
shardCount++ shardCount++
) )
); );

View File

@ -47,6 +47,9 @@ import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec; import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.server.DruidNode; import io.druid.server.DruidNode;
@ -68,7 +71,6 @@ import java.io.Reader;
import java.nio.charset.Charset; import java.nio.charset.Charset;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.SortedSet; import java.util.SortedSet;
@ -80,13 +82,13 @@ public class HadoopDruidIndexerConfig
private static final Injector injector; private static final Injector injector;
public static final String CONFIG_PROPERTY = "druid.indexer.config"; public static final String CONFIG_PROPERTY = "druid.indexer.config";
public static final Charset javaNativeCharset = Charset.forName("Unicode"); public static final Charset JAVA_NATIVE_CHARSET = Charset.forName("Unicode");
public static final Splitter tabSplitter = Splitter.on("\t"); public static final Splitter TAB_SPLITTER = Splitter.on("\t");
public static final Joiner tabJoiner = Joiner.on("\t"); public static final Joiner TAB_JOINER = Joiner.on("\t");
public static final ObjectMapper jsonMapper; public static final ObjectMapper JSON_MAPPER;
public static final IndexIO INDEX_IO;
// workaround to pass down druid.processing.bitmap.type, see IndexGeneratorJob.run() public static final IndexMerger INDEX_MERGER;
protected static final Properties properties; public static final IndexMaker INDEX_MAKER;
private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing";
@ -107,8 +109,10 @@ public class HadoopDruidIndexerConfig
new IndexingHadoopModule() new IndexingHadoopModule()
) )
); );
jsonMapper = injector.getInstance(ObjectMapper.class); JSON_MAPPER = injector.getInstance(ObjectMapper.class);
properties = injector.getInstance(Properties.class); INDEX_IO = injector.getInstance(IndexIO.class);
INDEX_MERGER = injector.getInstance(IndexMerger.class);
INDEX_MAKER = injector.getInstance(IndexMaker.class);
} }
public static enum IndexJobCounters public static enum IndexJobCounters
@ -127,13 +131,13 @@ public class HadoopDruidIndexerConfig
// the Map<> intermediary // the Map<> intermediary
if (argSpec.containsKey("spec")) { if (argSpec.containsKey("spec")) {
return HadoopDruidIndexerConfig.jsonMapper.convertValue( return HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
argSpec, argSpec,
HadoopDruidIndexerConfig.class HadoopDruidIndexerConfig.class
); );
} }
return new HadoopDruidIndexerConfig( return new HadoopDruidIndexerConfig(
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
argSpec, argSpec,
HadoopIngestionSpec.class HadoopIngestionSpec.class
) )
@ -145,7 +149,7 @@ public class HadoopDruidIndexerConfig
{ {
try { try {
return fromMap( return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.jsonMapper.readValue( (Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
file, new TypeReference<Map<String, Object>>() file, new TypeReference<Map<String, Object>>()
{ {
} }
@ -163,7 +167,7 @@ public class HadoopDruidIndexerConfig
// This is a map to try and prevent dependency screwbally-ness // This is a map to try and prevent dependency screwbally-ness
try { try {
return fromMap( return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.jsonMapper.readValue( (Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
str, new TypeReference<Map<String, Object>>() str, new TypeReference<Map<String, Object>>()
{ {
} }
@ -185,12 +189,12 @@ public class HadoopDruidIndexerConfig
Reader reader = new InputStreamReader(fs.open(pt)); Reader reader = new InputStreamReader(fs.open(pt));
return fromMap( return fromMap(
(Map<String, Object>) HadoopDruidIndexerConfig.jsonMapper.readValue( (Map<String, Object>) HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
reader, new TypeReference<Map<String, Object>>() reader, new TypeReference<Map<String, Object>>()
{ {
} }
) )
); );
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -216,7 +220,7 @@ public class HadoopDruidIndexerConfig
) )
{ {
this.schema = spec; this.schema = spec;
this.pathSpec = jsonMapper.convertValue(spec.getIOConfig().getPathSpec(), PathSpec.class); this.pathSpec = JSON_MAPPER.convertValue(spec.getIOConfig().getPathSpec(), PathSpec.class);
for (Map.Entry<DateTime, List<HadoopyShardSpec>> entry : spec.getTuningConfig().getShardSpecs().entrySet()) { for (Map.Entry<DateTime, List<HadoopyShardSpec>> entry : spec.getTuningConfig().getShardSpecs().entrySet()) {
if (entry.getValue() == null || entry.getValue().isEmpty()) { if (entry.getValue() == null || entry.getValue().isEmpty()) {
continue; continue;
@ -268,7 +272,7 @@ public class HadoopDruidIndexerConfig
public void setGranularitySpec(GranularitySpec granularitySpec) public void setGranularitySpec(GranularitySpec granularitySpec)
{ {
this.schema = schema.withDataSchema(schema.getDataSchema().withGranularitySpec(granularitySpec)); this.schema = schema.withDataSchema(schema.getDataSchema().withGranularitySpec(granularitySpec));
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); this.pathSpec = JSON_MAPPER.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
} }
public PartitionsSpec getPartitionsSpec() public PartitionsSpec getPartitionsSpec()
@ -294,13 +298,13 @@ public class HadoopDruidIndexerConfig
public void setVersion(String version) public void setVersion(String version)
{ {
this.schema = schema.withTuningConfig(schema.getTuningConfig().withVersion(version)); this.schema = schema.withTuningConfig(schema.getTuningConfig().withVersion(version));
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); this.pathSpec = JSON_MAPPER.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
} }
public void setShardSpecs(Map<DateTime, List<HadoopyShardSpec>> shardSpecs) public void setShardSpecs(Map<DateTime, List<HadoopyShardSpec>> shardSpecs)
{ {
this.schema = schema.withTuningConfig(schema.getTuningConfig().withShardSpecs(shardSpecs)); this.schema = schema.withTuningConfig(schema.getTuningConfig().withShardSpecs(shardSpecs));
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); this.pathSpec = JSON_MAPPER.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
} }
public Optional<List<Interval>> getIntervals() public Optional<List<Interval>> getIntervals()
@ -536,7 +540,7 @@ public class HadoopDruidIndexerConfig
Configuration conf = job.getConfiguration(); Configuration conf = job.getConfiguration();
try { try {
conf.set(HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(this)); conf.set(HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(this));
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -546,7 +550,7 @@ public class HadoopDruidIndexerConfig
public void verify() public void verify()
{ {
try { try {
log.info("Running with config:%n%s", jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(this)); log.info("Running with config:%n%s", JSON_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(this));
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -19,7 +19,6 @@ package io.druid.indexer;
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.Throwables; import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
@ -38,8 +37,6 @@ import io.druid.data.input.Rows;
import io.druid.indexer.hadoop.SegmentInputRow; import io.druid.indexer.hadoop.SegmentInputRow;
import io.druid.offheap.OffheapBufferPool; import io.druid.offheap.OffheapBufferPool;
import io.druid.query.aggregation.AggregatorFactory; 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.LoggingProgressIndicator;
import io.druid.segment.ProgressIndicator; import io.druid.segment.ProgressIndicator;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
@ -85,7 +82,7 @@ public class IndexGeneratorJob implements Jobby
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config) public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
{ {
final Configuration conf = JobHelper.injectSystemProperties(new Configuration()); final Configuration conf = JobHelper.injectSystemProperties(new Configuration());
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER;
ImmutableList.Builder<DataSegment> publishedSegmentsBuilder = ImmutableList.builder(); ImmutableList.Builder<DataSegment> publishedSegmentsBuilder = ImmutableList.builder();
@ -176,18 +173,6 @@ public class IndexGeneratorJob implements Jobby
config.addInputPaths(job); config.addInputPaths(job);
// hack to get druid.processing.bitmap property passed down to hadoop job.
// once IndexIO doesn't rely on globally injected properties, we can move this into the HadoopTuningConfig.
final String bitmapProperty = "druid.processing.bitmap.type";
final String bitmapType = HadoopDruidIndexerConfig.properties.getProperty(bitmapProperty);
if (bitmapType != null) {
for (String property : new String[]{"mapreduce.reduce.java.opts", "mapreduce.map.java.opts"}) {
// prepend property to allow overriding using hadoop.xxx properties by JobHelper.injectSystemProperties above
String value = Strings.nullToEmpty(job.getConfiguration().get(property));
job.getConfiguration().set(property, String.format("-D%s=%s %s", bitmapProperty, bitmapType, value));
}
}
config.intoConfiguration(job); config.intoConfiguration(job);
JobHelper.setupClasspath( JobHelper.setupClasspath(
@ -277,7 +262,7 @@ public class IndexGeneratorJob implements Jobby
final long truncatedTimestamp = granularitySpec.getQueryGranularity().truncate(inputRow.getTimestampFromEpoch()); final long truncatedTimestamp = granularitySpec.getQueryGranularity().truncate(inputRow.getTimestampFromEpoch());
final byte[] hashedDimensions = hashFunction.hashBytes( final byte[] hashedDimensions = hashFunction.hashBytes(
HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes( HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(
Rows.toGroupKey( Rows.toGroupKey(
truncatedTimestamp, truncatedTimestamp,
inputRow inputRow
@ -489,7 +474,7 @@ public class IndexGeneratorJob implements Jobby
final ProgressIndicator progressIndicator final ProgressIndicator progressIndicator
) throws IOException ) throws IOException
{ {
return IndexMaker.persist( return HadoopDruidIndexerConfig.INDEX_MAKER.persist(
index, interval, file, null, config.getIndexSpec(), progressIndicator index, interval, file, null, config.getIndexSpec(), progressIndicator
); );
} }
@ -501,7 +486,7 @@ public class IndexGeneratorJob implements Jobby
ProgressIndicator progressIndicator ProgressIndicator progressIndicator
) throws IOException ) throws IOException
{ {
return IndexMaker.mergeQueryableIndex( return HadoopDruidIndexerConfig.INDEX_MAKER.mergeQueryableIndex(
indexes, aggs, file, config.getIndexSpec(), progressIndicator indexes, aggs, file, config.getIndexSpec(), progressIndicator
); );
} }
@ -614,7 +599,7 @@ public class IndexGeneratorJob implements Jobby
} }
for (File file : toMerge) { for (File file : toMerge) {
indexes.add(IndexIO.loadIndex(file)); indexes.add(HadoopDruidIndexerConfig.INDEX_IO.loadIndex(file));
} }
mergedBase = mergeQueryableIndex( mergedBase = mergeQueryableIndex(
indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator

View File

@ -462,7 +462,7 @@ public class JobHelper
DEFAULT_FS_BUFFER_SIZE, DEFAULT_FS_BUFFER_SIZE,
progressable progressable
)) { )) {
HadoopDruidIndexerConfig.jsonMapper.writeValue(descriptorOut, segment); HadoopDruidIndexerConfig.JSON_MAPPER.writeValue(descriptorOut, segment);
descriptorOut.flush(); descriptorOut.flush();
} }
} }

View File

@ -19,8 +19,6 @@ 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.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;
@ -68,7 +66,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, null, config.getIndexSpec(), progressIndicator); return HadoopDruidIndexerConfig.INDEX_MERGER.persist(index, interval, file, null, config.getIndexSpec(), progressIndicator);
} }
@Override @Override
@ -79,7 +77,7 @@ public class LegacyIndexGeneratorJob extends IndexGeneratorJob
ProgressIndicator progressIndicator ProgressIndicator progressIndicator
) throws IOException ) throws IOException
{ {
return IndexMerger.mergeQueryableIndex(indexes, aggs, file, config.getIndexSpec(), progressIndicator); return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex(indexes, aggs, file, config.getIndexSpec(), progressIndicator);
} }
} }
} }

View File

@ -42,7 +42,7 @@ public class MetadataStorageUpdaterJob implements Jobby
{ {
final List<DataSegment> segments = IndexGeneratorJob.getPublishedSegments(config); final List<DataSegment> segments = IndexGeneratorJob.getPublishedSegments(config);
final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable(); final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable();
handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.jsonMapper); handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.JSON_MAPPER);
return true; return true;
} }

View File

@ -26,7 +26,6 @@ import com.metamx.common.ISE;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.timeline.DataSegment;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputFormat;
@ -55,7 +54,7 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
Configuration conf = context.getConfiguration(); Configuration conf = context.getConfiguration();
String segmentsStr = Preconditions.checkNotNull(conf.get(CONF_INPUT_SEGMENTS), "No segments found to read"); String segmentsStr = Preconditions.checkNotNull(conf.get(CONF_INPUT_SEGMENTS), "No segments found to read");
List<WindowedDataSegment> segments = HadoopDruidIndexerConfig.jsonMapper.readValue( List<WindowedDataSegment> segments = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
segmentsStr, segmentsStr,
new TypeReference<List<WindowedDataSegment>>() new TypeReference<List<WindowedDataSegment>>()
{ {

View File

@ -70,13 +70,13 @@ public class DatasourceInputSplit extends InputSplit implements Writable
@Override @Override
public void write(DataOutput out) throws IOException public void write(DataOutput out) throws IOException
{ {
out.writeUTF(HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segments)); out.writeUTF(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(segments));
} }
@Override @Override
public void readFields(DataInput in) throws IOException public void readFields(DataInput in) throws IOException
{ {
segments = HadoopDruidIndexerConfig.jsonMapper.readValue( segments = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
in.readUTF(), in.readUTF(),
new TypeReference<List<WindowedDataSegment>>() new TypeReference<List<WindowedDataSegment>>()
{ {

View File

@ -33,7 +33,6 @@ import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.MapBasedRow; import io.druid.data.input.MapBasedRow;
import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.segment.IndexIO;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.QueryableIndexStorageAdapter;
import io.druid.segment.realtime.firehose.IngestSegmentFirehose; import io.druid.segment.realtime.firehose.IngestSegmentFirehose;
@ -67,7 +66,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
@Override @Override
public void initialize(InputSplit split, final TaskAttemptContext context) throws IOException, InterruptedException public void initialize(InputSplit split, final TaskAttemptContext context) throws IOException, InterruptedException
{ {
spec = readAndVerifyDatasourceIngestionSpec(context.getConfiguration(), HadoopDruidIndexerConfig.jsonMapper); spec = readAndVerifyDatasourceIngestionSpec(context.getConfiguration(), HadoopDruidIndexerConfig.JSON_MAPPER);
List<WindowedDataSegment> segments = ((DatasourceInputSplit) split).getSegments(); List<WindowedDataSegment> segments = ((DatasourceInputSplit) split).getSegments();
@ -91,7 +90,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
JobHelper.unzipNoGuava(path, context.getConfiguration(), dir, context); JobHelper.unzipNoGuava(path, context.getConfiguration(), dir, context);
logger.info("finished fetching segment files"); logger.info("finished fetching segment files");
QueryableIndex index = IndexIO.loadIndex(dir); QueryableIndex index = HadoopDruidIndexerConfig.INDEX_IO.loadIndex(dir);
indexes.add(index); indexes.add(index);
numRows += index.getNumRows(); numRows += index.getNumRows();

View File

@ -34,8 +34,6 @@ import com.metamx.common.logger.Logger;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.indexer.hadoop.DatasourceInputSplit; import io.druid.indexer.hadoop.DatasourceInputSplit;
import io.druid.indexer.hadoop.WindowedDataSegment; import io.druid.indexer.hadoop.WindowedDataSegment;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -514,7 +512,7 @@ public class HadoopConverterJob
if (!outDir.mkdir() && (!outDir.exists() || !outDir.isDirectory())) { if (!outDir.mkdir() && (!outDir.exists() || !outDir.isDirectory())) {
throw new IOException(String.format("Could not create output directory [%s]", outDir)); throw new IOException(String.format("Could not create output directory [%s]", outDir));
} }
IndexMerger.convert( HadoopDruidConverterConfig.INDEX_MERGER.convert(
inDir, inDir,
outDir, outDir,
config.getIndexSpec(), config.getIndexSpec(),
@ -522,7 +520,7 @@ public class HadoopConverterJob
); );
if (config.isValidate()) { if (config.isValidate()) {
context.setStatus("Validating"); context.setStatus("Validating");
IndexIO.DefaultIndexIOHandler.validateTwoSegments(inDir, outDir); HadoopDruidConverterConfig.INDEX_IO.validateTwoSegments(inDir, outDir);
} }
context.progress(); context.progress();
context.setStatus("Starting PUSH"); context.setStatus("Starting PUSH");

View File

@ -34,6 +34,8 @@ import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider; import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self; import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.server.DruidNode; import io.druid.server.DruidNode;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -49,6 +51,9 @@ public class HadoopDruidConverterConfig
{ {
public static final String CONFIG_PROPERTY = "io.druid.indexer.updater.converter"; public static final String CONFIG_PROPERTY = "io.druid.indexer.updater.converter";
public static final ObjectMapper jsonMapper; public static final ObjectMapper jsonMapper;
public static final IndexIO INDEX_IO;
public static final IndexMerger INDEX_MERGER;
private static final Injector injector = Initialization.makeInjectorWithModules( private static final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(), GuiceInjectors.makeStartupInjector(),
ImmutableList.<Module>of( ImmutableList.<Module>of(
@ -68,6 +73,8 @@ public class HadoopDruidConverterConfig
static { static {
jsonMapper = injector.getInstance(ObjectMapper.class); jsonMapper = injector.getInstance(ObjectMapper.class);
jsonMapper.registerSubtypes(HadoopDruidConverterConfig.class); jsonMapper.registerSubtypes(HadoopDruidConverterConfig.class);
INDEX_IO = injector.getInstance(IndexIO.class);
INDEX_MERGER = injector.getInstance(IndexMerger.class);
} }
private static final TypeReference<Map<String, Object>> mapTypeReference = new TypeReference<Map<String, Object>>() private static final TypeReference<Map<String, Object>> mapTypeReference = new TypeReference<Map<String, Object>>()

View File

@ -70,6 +70,7 @@ public class BatchDeltaIngestionTest
public final TemporaryFolder temporaryFolder = new TemporaryFolder(); public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private static final ObjectMapper MAPPER; private static final ObjectMapper MAPPER;
private static final IndexIO INDEX_IO;
private static final Interval INTERVAL_FULL = new Interval("2014-10-22T00:00:00Z/P1D"); private static final Interval INTERVAL_FULL = new Interval("2014-10-22T00:00:00Z/P1D");
private static final Interval INTERVAL_PARTIAL = new Interval("2014-10-22T00:00:00Z/PT2H"); private static final Interval INTERVAL_PARTIAL = new Interval("2014-10-22T00:00:00Z/PT2H");
private static final DataSegment SEGMENT; private static final DataSegment SEGMENT;
@ -79,6 +80,7 @@ public class BatchDeltaIngestionTest
MAPPER.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); MAPPER.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed"));
InjectableValues inject = new InjectableValues.Std().addValue(ObjectMapper.class, MAPPER); InjectableValues inject = new InjectableValues.Std().addValue(ObjectMapper.class, MAPPER);
MAPPER.setInjectableValues(inject); MAPPER.setInjectableValues(inject);
INDEX_IO = HadoopDruidIndexerConfig.INDEX_IO;
try { try {
SEGMENT = new DefaultObjectMapper() SEGMENT = new DefaultObjectMapper()
@ -314,7 +316,7 @@ public class BatchDeltaIngestionTest
File tmpUnzippedSegmentDir = temporaryFolder.newFolder(); File tmpUnzippedSegmentDir = temporaryFolder.newFolder();
new LocalDataSegmentPuller().getSegmentFiles(dataSegment, tmpUnzippedSegmentDir); new LocalDataSegmentPuller().getSegmentFiles(dataSegment, tmpUnzippedSegmentDir);
QueryableIndex index = IndexIO.loadIndex(tmpUnzippedSegmentDir); QueryableIndex index = INDEX_IO.loadIndex(tmpUnzippedSegmentDir);
StorageAdapter adapter = new QueryableIndexStorageAdapter(index); StorageAdapter adapter = new QueryableIndexStorageAdapter(index);
Firehose firehose = new IngestSegmentFirehose( Firehose firehose = new IngestSegmentFirehose(
@ -392,7 +394,7 @@ public class BatchDeltaIngestionTest
INTERVAL_FULL.getStart(), INTERVAL_FULL.getStart(),
ImmutableList.of( ImmutableList.of(
new HadoopyShardSpec( new HadoopyShardSpec(
new HashBasedNumberedShardSpec(0, 1, HadoopDruidIndexerConfig.jsonMapper), new HashBasedNumberedShardSpec(0, 1, HadoopDruidIndexerConfig.JSON_MAPPER),
0 0
) )
) )

View File

@ -108,7 +108,7 @@ public class DetermineHashedPartitionsJobTest
HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec( HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec(
new DataSchema( new DataSchema(
"test_schema", "test_schema",
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new DelimitedParseSpec( new DelimitedParseSpec(
new TimestampSpec("ts", null, null), new TimestampSpec("ts", null, null),
@ -137,7 +137,7 @@ public class DetermineHashedPartitionsJobTest
QueryGranularity.NONE, QueryGranularity.NONE,
ImmutableList.of(new Interval(interval)) ImmutableList.of(new Interval(interval))
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(

View File

@ -223,7 +223,7 @@ public class DeterminePartitionsJobTest
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( new DataSchema(
"website", "website",
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
@ -238,7 +238,7 @@ public class DeterminePartitionsJobTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(new Interval(interval)) Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(new Interval(interval))
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(

View File

@ -37,7 +37,7 @@ public class HadoopyStringInputRowParserTest
+ "\"parseSpec\":{\"format\":\"json\",\"timestampSpec\":{\"column\":\"xXx\"},\"dimensionsSpec\":{}}" + "\"parseSpec\":{\"format\":\"json\",\"timestampSpec\":{\"column\":\"xXx\"},\"dimensionsSpec\":{}}"
+ "}"; + "}";
ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper; ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER;
InputRowParser parser = jsonMapper.readValue( InputRowParser parser = jsonMapper.readValue(
jsonMapper.writeValueAsString( jsonMapper.writeValueAsString(
jsonMapper.readValue(jsonStr, InputRowParser.class) jsonMapper.readValue(jsonStr, InputRowParser.class)

View File

@ -63,7 +63,7 @@ public class IndexGeneratorCombinerTest
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( new DataSchema(
"website", "website",
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
@ -81,7 +81,7 @@ public class IndexGeneratorCombinerTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2010/2011")) Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2010/2011"))
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(
@ -99,7 +99,7 @@ public class IndexGeneratorCombinerTest
Configuration hadoopConfig = new Configuration(); Configuration hadoopConfig = new Configuration();
hadoopConfig.set( hadoopConfig.set(
HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.CONFIG_PROPERTY,
HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(config) HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config)
); );
Reducer.Context context = EasyMock.createMock(Reducer.Context.class); Reducer.Context context = EasyMock.createMock(Reducer.Context.class);

View File

@ -339,7 +339,7 @@ public class IndexGeneratorJobTest
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
{ {
mapper = HadoopDruidIndexerConfig.jsonMapper; mapper = HadoopDruidIndexerConfig.JSON_MAPPER;
mapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); mapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed"));
mapper.registerSubtypes(new NamedType(SingleDimensionShardSpec.class, "single")); mapper.registerSubtypes(new NamedType(SingleDimensionShardSpec.class, "single"));
@ -412,7 +412,7 @@ public class IndexGeneratorJobTest
List<ShardSpec> specs = Lists.newArrayList(); List<ShardSpec> specs = Lists.newArrayList();
if (partitionType.equals("hashed")) { if (partitionType.equals("hashed")) {
for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) { for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) {
specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], HadoopDruidIndexerConfig.jsonMapper)); specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], HadoopDruidIndexerConfig.JSON_MAPPER));
} }
} else if (partitionType.equals("single")) { } else if (partitionType.equals("single")) {
int partitionNum = 0; int partitionNum = 0;

View File

@ -66,7 +66,7 @@ public class JobHelperTest
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( new DataSchema(
"website", "website",
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
@ -81,7 +81,7 @@ public class JobHelperTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(this.interval) Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(this.interval)
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(

View File

@ -60,7 +60,7 @@ public class DatasourceRecordReaderTest
Configuration config = new Configuration(); Configuration config = new Configuration();
config.set( config.set(
DatasourceInputFormat.CONF_DRUID_SCHEMA, DatasourceInputFormat.CONF_DRUID_SCHEMA,
HadoopDruidIndexerConfig.jsonMapper.writeValueAsString( HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(
new DatasourceIngestionSpec( new DatasourceIngestionSpec(
segment.getDataSource(), segment.getDataSource(),
segment.getInterval(), segment.getInterval(),

View File

@ -177,7 +177,7 @@ public class DatasourcePathSpecTest
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( new DataSchema(
ingestionSpec.getDataSource(), ingestionSpec.getDataSource(),
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
@ -194,7 +194,7 @@ public class DatasourcePathSpecTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2000/3000")) Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2000/3000"))
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(

View File

@ -156,7 +156,7 @@ public class HadoopConverterJobTest
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( new DataSchema(
DATASOURCE, DATASOURCE,
HadoopDruidIndexerConfig.jsonMapper.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new DelimitedParseSpec( new DelimitedParseSpec(
new TimestampSpec("ts", "iso", null), new TimestampSpec("ts", "iso", null),
@ -177,7 +177,7 @@ public class HadoopConverterJobTest
QueryGranularity.DAY, QueryGranularity.DAY,
ImmutableList.<Interval>of(interval) ImmutableList.<Interval>of(interval)
), ),
HadoopDruidIndexerConfig.jsonMapper HadoopDruidIndexerConfig.JSON_MAPPER
), ),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of(

View File

@ -19,6 +19,7 @@ package io.druid.indexing.common;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Multimap; import com.google.common.collect.Multimap;
@ -32,6 +33,9 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentMover;
@ -70,6 +74,9 @@ public class TaskToolbox
private final SegmentLoader segmentLoader; private final SegmentLoader segmentLoader;
private final ObjectMapper objectMapper; private final ObjectMapper objectMapper;
private final File taskWorkDir; private final File taskWorkDir;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
public TaskToolbox( public TaskToolbox(
TaskConfig config, TaskConfig config,
@ -87,7 +94,10 @@ public class TaskToolbox
MonitorScheduler monitorScheduler, MonitorScheduler monitorScheduler,
SegmentLoader segmentLoader, SegmentLoader segmentLoader,
ObjectMapper objectMapper, ObjectMapper objectMapper,
final File taskWorkDir File taskWorkDir,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
) )
{ {
this.config = config; this.config = config;
@ -106,6 +116,9 @@ public class TaskToolbox
this.segmentLoader = segmentLoader; this.segmentLoader = segmentLoader;
this.objectMapper = objectMapper; this.objectMapper = objectMapper;
this.taskWorkDir = taskWorkDir; this.taskWorkDir = taskWorkDir;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexMaker = Preconditions.checkNotNull(indexMaker, "Null IndexMaker");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
} }
public TaskConfig getConfig() public TaskConfig getConfig()
@ -208,4 +221,19 @@ public class TaskToolbox
{ {
return taskWorkDir; return taskWorkDir;
} }
public IndexIO getIndexIO()
{
return indexIO;
}
public IndexMerger getIndexMerger()
{
return indexMerger;
}
public IndexMaker getIndexMaker()
{
return indexMaker;
}
} }

View File

@ -18,6 +18,7 @@
package io.druid.indexing.common; package io.druid.indexing.common;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorScheduler;
@ -27,6 +28,9 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentMover;
@ -55,6 +59,9 @@ public class TaskToolboxFactory
private final MonitorScheduler monitorScheduler; private final MonitorScheduler monitorScheduler;
private final SegmentLoaderFactory segmentLoaderFactory; private final SegmentLoaderFactory segmentLoaderFactory;
private final ObjectMapper objectMapper; private final ObjectMapper objectMapper;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@Inject @Inject
public TaskToolboxFactory( public TaskToolboxFactory(
@ -71,7 +78,10 @@ public class TaskToolboxFactory
@Processing ExecutorService queryExecutorService, @Processing ExecutorService queryExecutorService,
MonitorScheduler monitorScheduler, MonitorScheduler monitorScheduler,
SegmentLoaderFactory segmentLoaderFactory, SegmentLoaderFactory segmentLoaderFactory,
ObjectMapper objectMapper ObjectMapper objectMapper,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
) )
{ {
this.config = config; this.config = config;
@ -88,6 +98,9 @@ public class TaskToolboxFactory
this.monitorScheduler = monitorScheduler; this.monitorScheduler = monitorScheduler;
this.segmentLoaderFactory = segmentLoaderFactory; this.segmentLoaderFactory = segmentLoaderFactory;
this.objectMapper = objectMapper; this.objectMapper = objectMapper;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexMaker = Preconditions.checkNotNull(indexMaker, "Null IndexMaker");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
} }
public TaskToolbox build(Task task) public TaskToolbox build(Task task)
@ -110,7 +123,10 @@ public class TaskToolboxFactory
monitorScheduler, monitorScheduler,
segmentLoaderFactory.manufacturate(taskWorkDir), segmentLoaderFactory.manufacturate(taskWorkDir),
objectMapper, objectMapper,
taskWorkDir taskWorkDir,
indexMerger,
indexMaker,
indexIO
); );
} }
} }

View File

@ -65,6 +65,8 @@ public class YeOldePlumberSchool implements PlumberSchool
private final String version; private final String version;
private final DataSegmentPusher dataSegmentPusher; private final DataSegmentPusher dataSegmentPusher;
private final File tmpSegmentDir; private final File tmpSegmentDir;
private final IndexMerger indexMerger;
private final IndexIO indexIO;
private static final Logger log = new Logger(YeOldePlumberSchool.class); private static final Logger log = new Logger(YeOldePlumberSchool.class);
@ -73,13 +75,17 @@ public class YeOldePlumberSchool implements PlumberSchool
@JsonProperty("interval") Interval interval, @JsonProperty("interval") Interval interval,
@JsonProperty("version") String version, @JsonProperty("version") String version,
@JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher,
@JacksonInject("tmpSegmentDir") File tmpSegmentDir @JacksonInject("tmpSegmentDir") File tmpSegmentDir,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexIO indexIO
) )
{ {
this.interval = interval; this.interval = interval;
this.version = version; this.version = version;
this.dataSegmentPusher = dataSegmentPusher; this.dataSegmentPusher = dataSegmentPusher;
this.tmpSegmentDir = tmpSegmentDir; this.tmpSegmentDir = tmpSegmentDir;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
} }
@Override @Override
@ -162,15 +168,15 @@ public class YeOldePlumberSchool implements PlumberSchool
} else { } else {
List<QueryableIndex> indexes = Lists.newArrayList(); List<QueryableIndex> indexes = Lists.newArrayList();
for (final File oneSpill : spilled) { for (final File oneSpill : spilled) {
indexes.add(IndexIO.loadIndex(oneSpill)); indexes.add(indexIO.loadIndex(oneSpill));
} }
fileToUpload = new File(tmpSegmentDir, "merged"); fileToUpload = new File(tmpSegmentDir, "merged");
IndexMerger.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload, config.getIndexSpec()); indexMerger.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload, config.getIndexSpec());
} }
// Map merged segment so we can extract dimensions // Map merged segment so we can extract dimensions
final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload); final QueryableIndex mappedSegment = indexIO.loadIndex(fileToUpload);
final DataSegment segmentToUpload = theSink.getSegment() final DataSegment segmentToUpload = theSink.getSegment()
.withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions()))
@ -211,7 +217,7 @@ public class YeOldePlumberSchool implements PlumberSchool
log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist); log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist);
try { try {
IndexMerger.persist( indexMerger.persist(
indexToPersist.getIndex(), indexToPersist.getIndex(),
dirToPersist, dirToPersist,
null, null,

View File

@ -25,8 +25,7 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import io.druid.segment.IndexIO; import io.druid.indexing.common.TaskToolbox;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; 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;
@ -64,7 +63,7 @@ public class AppendTask extends MergeTaskBase
} }
@Override @Override
public File merge(final Map<DataSegment, File> segments, final File outDir) public File merge(final TaskToolbox toolbox, final Map<DataSegment, File> segments, final File outDir)
throws Exception throws Exception
{ {
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>( VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
@ -113,7 +112,7 @@ public class AppendTask extends MergeTaskBase
adapters.add( adapters.add(
new RowboatFilteringIndexAdapter( new RowboatFilteringIndexAdapter(
new QueryableIndexIndexableAdapter( new QueryableIndexIndexableAdapter(
IndexIO.loadIndex(holder.getFile()) toolbox.getIndexIO().loadIndex(holder.getFile())
), ),
new Predicate<Rowboat>() new Predicate<Rowboat>()
{ {
@ -127,7 +126,7 @@ public class AppendTask extends MergeTaskBase
); );
} }
return IndexMerger.append(adapters, outDir, indexSpec); return toolbox.getIndexMerger().append(adapters, outDir, indexSpec);
} }
@Override @Override

View File

@ -39,7 +39,6 @@ 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;
import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
@ -99,7 +98,12 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
* *
* @return A SegmentConverterTask for the segment with the indexSpec specified. * @return A SegmentConverterTask for the segment with the indexSpec specified.
*/ */
public static ConvertSegmentTask create(DataSegment segment, IndexSpec indexSpec, boolean force, boolean validate, Map<String, Object> context public static ConvertSegmentTask create(
DataSegment segment,
IndexSpec indexSpec,
boolean force,
boolean validate,
Map<String, Object> context
) )
{ {
final Interval interval = segment.getInterval(); final Interval interval = segment.getInterval();
@ -377,7 +381,7 @@ public class ConvertSegmentTask 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, indexSpec, force, validate)) { if (toolbox.getIndexIO().convertSegment(location, outLocation, indexSpec, force, validate)) {
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

View File

@ -234,7 +234,7 @@ public class HadoopIndexTask extends HadoopTask
final String schema = args[0]; final String schema = args[0];
String version = args[1]; String version = args[1];
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.jsonMapper final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER
.readValue( .readValue(
schema, schema,
HadoopIngestionSpec.class HadoopIngestionSpec.class
@ -257,7 +257,7 @@ public class HadoopIndexTask extends HadoopTask
log.info("Starting a hadoop index generator job..."); log.info("Starting a hadoop index generator job...");
if (job.run()) { if (job.run()) {
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(job.getPublishedSegments()); return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(job.getPublishedSegments());
} }
return null; return null;
@ -272,7 +272,7 @@ public class HadoopIndexTask extends HadoopTask
final String workingPath = args[1]; final String workingPath = args[1];
final String segmentOutputPath = args[2]; final String segmentOutputPath = args[2];
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.jsonMapper final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER
.readValue( .readValue(
schema, schema,
HadoopIngestionSpec.class HadoopIngestionSpec.class
@ -287,7 +287,7 @@ public class HadoopIndexTask extends HadoopTask
log.info("Starting a hadoop determine configuration job..."); log.info("Starting a hadoop determine configuration job...");
if (job.run()) { if (job.run()) {
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(config.getSchema()); return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config.getSchema());
} }
return null; return null;

View File

@ -61,7 +61,6 @@ import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.ShardSpec; import io.druid.timeline.partition.ShardSpec;
import java.util.Map;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -69,6 +68,7 @@ import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
@ -359,7 +359,9 @@ public class IndexTask extends AbstractFixedIntervalTask
interval, interval,
version, version,
wrappedDataSegmentPusher, wrappedDataSegmentPusher,
tmpDir tmpDir,
toolbox.getIndexMerger(),
toolbox.getIndexIO()
).findPlumber( ).findPlumber(
schema, schema,
convertTuningConfig(shardSpec, myRowFlushBoundary, ingestionSchema.getTuningConfig().getIndexSpec()), convertTuningConfig(shardSpec, myRowFlushBoundary, ingestionSchema.getTuningConfig().getIndexSpec()),

View File

@ -25,9 +25,8 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.indexing.common.TaskToolbox;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; 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;
@ -53,7 +52,6 @@ public class MergeTask extends MergeTaskBase
@JsonProperty("aggregations") List<AggregatorFactory> aggregators, @JsonProperty("aggregations") List<AggregatorFactory> aggregators,
@JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("context") Map<String, Object> context @JsonProperty("context") Map<String, Object> context
) )
{ {
super(id, dataSource, segments, context); super(id, dataSource, segments, context);
@ -62,10 +60,10 @@ public class MergeTask extends MergeTaskBase
} }
@Override @Override
public File merge(final Map<DataSegment, File> segments, final File outDir) public File merge(final TaskToolbox toolbox, final Map<DataSegment, File> segments, final File outDir)
throws Exception throws Exception
{ {
return IndexMerger.mergeQueryableIndex( return toolbox.getIndexMerger().mergeQueryableIndex(
Lists.transform( Lists.transform(
ImmutableList.copyOf(segments.values()), ImmutableList.copyOf(segments.values()),
new Function<File, QueryableIndex>() new Function<File, QueryableIndex>()
@ -74,7 +72,7 @@ public class MergeTask extends MergeTaskBase
public QueryableIndex apply(@Nullable File input) public QueryableIndex apply(@Nullable File input)
{ {
try { try {
return IndexIO.loadIndex(input); return toolbox.getIndexIO().loadIndex(input);
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -149,7 +149,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
final Map<DataSegment, File> gettedSegments = toolbox.fetchSegments(segments); final Map<DataSegment, File> gettedSegments = toolbox.fetchSegments(segments);
// merge files together // merge files together
final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); final File fileToUpload = merge(toolbox, gettedSegments, new File(taskDir, "merged"));
emitter.emit(builder.build("merger/numMerged", segments.size())); emitter.emit(builder.build("merger/numMerged", segments.size()));
emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime));
@ -230,7 +230,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
} }
} }
protected abstract File merge(Map<DataSegment, File> segments, File outDir) protected abstract File merge(TaskToolbox taskToolbox, Map<DataSegment, File> segments, File outDir)
throws Exception; throws Exception;
@JsonProperty @JsonProperty

View File

@ -275,7 +275,10 @@ public class RealtimeIndexTask extends AbstractTask
lockingSegmentAnnouncer, lockingSegmentAnnouncer,
segmentPublisher, segmentPublisher,
toolbox.getNewSegmentServerView(), toolbox.getNewSegmentServerView(),
toolbox.getQueryExecutorService() toolbox.getQueryExecutorService(),
toolbox.getIndexMerger(),
toolbox.getIndexMaker(),
toolbox.getIndexIO()
); );
this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, fireDepartment.getMetrics()); this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, fireDepartment.getMetrics());

View File

@ -65,6 +65,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
private final List<String> dimensions; private final List<String> dimensions;
private final List<String> metrics; private final List<String> metrics;
private final Injector injector; private final Injector injector;
private final IndexIO indexIO;
@JsonCreator @JsonCreator
public IngestSegmentFirehoseFactory( public IngestSegmentFirehoseFactory(
@ -73,7 +74,8 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
@JsonProperty("filter") DimFilter dimFilter, @JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("dimensions") List<String> dimensions, @JsonProperty("dimensions") List<String> dimensions,
@JsonProperty("metrics") List<String> metrics, @JsonProperty("metrics") List<String> metrics,
@JacksonInject Injector injector @JacksonInject Injector injector,
@JacksonInject IndexIO indexIO
) )
{ {
Preconditions.checkNotNull(dataSource, "dataSource"); Preconditions.checkNotNull(dataSource, "dataSource");
@ -84,6 +86,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
this.dimensions = dimensions; this.dimensions = dimensions;
this.metrics = metrics; this.metrics = metrics;
this.injector = injector; this.injector = injector;
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
} }
@JsonProperty @JsonProperty
@ -247,7 +250,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
try { try {
return new WindowedStorageAdapter( return new WindowedStorageAdapter(
new QueryableIndexStorageAdapter( new QueryableIndexStorageAdapter(
IndexIO.loadIndex( indexIO.loadIndex(
Preconditions.checkNotNull( Preconditions.checkNotNull(
segmentFileMap.get(segment), segmentFileMap.get(segment),
"File for segment %s", segment.getIdentifier() "File for segment %s", segment.getIdentifier()

View File

@ -28,13 +28,16 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.Task;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentMover;
import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.DataSegmentPusher;
import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderConfig;
import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.easymock.EasyMock; import org.easymock.EasyMock;
@ -70,6 +73,9 @@ public class TaskToolboxTest
private ObjectMapper ObjectMapper = new ObjectMapper(); private ObjectMapper ObjectMapper = new ObjectMapper();
private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class); private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class);
private Task task = EasyMock.createMock(Task.class); private Task task = EasyMock.createMock(Task.class);
private IndexMerger mockIndexMerger = EasyMock.createMock(IndexMerger.class);
private IndexMaker mockIndexMaker = EasyMock.createMock(IndexMaker.class);
private IndexIO mockIndexIO = EasyMock.createMock(IndexIO.class);
@Rule @Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder(); public TemporaryFolder temporaryFolder = new TemporaryFolder();
@ -94,7 +100,10 @@ public class TaskToolboxTest
mockQueryExecutorService, mockQueryExecutorService,
mockMonitorScheduler, mockMonitorScheduler,
new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager), new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager),
ObjectMapper ObjectMapper,
mockIndexMerger,
mockIndexMaker,
mockIndexIO
); );
} }

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.RealtimeIndexTask;
import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.common.task.TaskResource;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeIOConfig;

View File

@ -17,8 +17,6 @@
package io.druid.indexing.common; package io.druid.indexing.common;
import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
@ -26,6 +24,10 @@ import com.google.common.base.Stopwatch;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import io.druid.guice.ServerModule; import io.druid.guice.ServerModule;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.column.ColumnConfig;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -34,28 +36,60 @@ import java.util.concurrent.TimeUnit;
*/ */
public class TestUtils public class TestUtils
{ {
public static final ObjectMapper MAPPER = new DefaultObjectMapper(); private final ObjectMapper jsonMapper;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
static { public TestUtils()
final List<? extends Module> list = new ServerModule().getJacksonModules(); {
for (Module module : list) { jsonMapper = new DefaultObjectMapper();
MAPPER.registerModule(module); indexIO = new IndexIO(
} jsonMapper,
MAPPER.setInjectableValues( new ColumnConfig()
new InjectableValues()
{ {
@Override @Override
public Object findInjectableValue( public int columnCacheSizeBytes()
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
{ {
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { return 0;
return TestUtils.MAPPER;
}
throw new ISE("No Injectable value found");
} }
} }
); );
indexMerger = new IndexMerger(jsonMapper, indexIO);
indexMaker = new IndexMaker(jsonMapper, indexIO);
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {
jsonMapper.registerModule(module);
}
jsonMapper.setInjectableValues(
new InjectableValues.Std()
.addValue(IndexIO.class, indexIO)
.addValue(IndexMerger.class, indexMerger)
.addValue(IndexMaker.class, indexMaker)
.addValue(ObjectMapper.class, jsonMapper)
);
}
public ObjectMapper getTestObjectMapper()
{
return jsonMapper;
}
public IndexMerger getTestIndexMerger()
{
return indexMerger;
}
public IndexMaker getTestIndexMaker()
{
return indexMaker;
}
public IndexIO getTestIndexIO()
{
return indexIO;
} }
public static boolean conditionValid(IndexingServiceCondition condition) public static boolean conditionValid(IndexingServiceCondition condition)

View File

@ -17,13 +17,12 @@
package io.druid.indexing.common.task; package io.druid.indexing.common.task;
import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.databind.ObjectMapper;
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 io.druid.jackson.DefaultObjectMapper; import io.druid.indexing.common.TestUtils;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import java.io.IOException;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
@ -33,7 +32,13 @@ import org.junit.Test;
*/ */
public class ConvertSegmentTaskTest public class ConvertSegmentTaskTest
{ {
private DefaultObjectMapper jsonMapper = new DefaultObjectMapper(); private final ObjectMapper jsonMapper;
public ConvertSegmentTaskTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test @Test
public void testSerializationSimple() throws Exception public void testSerializationSimple() throws Exception
@ -41,7 +46,6 @@ public class ConvertSegmentTaskTest
final String dataSource = "billy"; final String dataSource = "billy";
final Interval interval = new Interval(new DateTime().minus(1000), new DateTime()); final Interval interval = new Interval(new DateTime().minus(1000), new DateTime());
ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null); ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null);
Task task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class); Task task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class);

View File

@ -22,7 +22,7 @@ package io.druid.indexing.common.task;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
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 io.druid.jackson.DefaultObjectMapper; import io.druid.indexing.common.TestUtils;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
@ -39,7 +39,7 @@ import java.util.Map;
public class HadoopConverterTaskSerDeTest public class HadoopConverterTaskSerDeTest
{ {
private static ObjectMapper objectMapper = new DefaultObjectMapper();
private static final String TASK_ID = "task id"; private static final String TASK_ID = "task id";
private static final String DATA_SOURCE = "datasource"; private static final String DATA_SOURCE = "datasource";
private static final Interval INTERVAL = Interval.parse("2010/2011"); private static final Interval INTERVAL = Interval.parse("2010/2011");
@ -68,6 +68,14 @@ public class HadoopConverterTaskSerDeTest
private static final String OUTPUT_PATH = "/dev/null"; private static final String OUTPUT_PATH = "/dev/null";
private static final String CLASSPATH_PREFIX = "something:where:I:need:stuff"; private static final String CLASSPATH_PREFIX = "something:where:I:need:stuff";
private final ObjectMapper jsonMapper;
public HadoopConverterTaskSerDeTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test @Test
public void testSimpleConverterTaskSerDe() throws IOException public void testSimpleConverterTaskSerDe() throws IOException
{ {
@ -85,9 +93,9 @@ public class HadoopConverterTaskSerDeTest
CLASSPATH_PREFIX, CLASSPATH_PREFIX,
null null
); );
final String strOrig = objectMapper.writeValueAsString(orig); final String strOrig = jsonMapper.writeValueAsString(orig);
HadoopConverterTask other = objectMapper.readValue(strOrig, HadoopConverterTask.class); HadoopConverterTask other = jsonMapper.readValue(strOrig, HadoopConverterTask.class);
Assert.assertEquals(strOrig, objectMapper.writeValueAsString(other)); Assert.assertEquals(strOrig, jsonMapper.writeValueAsString(other));
Assert.assertFalse(orig == other); Assert.assertFalse(orig == other);
Assert.assertEquals(orig, other); Assert.assertEquals(orig, other);
assertExpectedTask(other); assertExpectedTask(other);
@ -117,13 +125,13 @@ public class HadoopConverterTaskSerDeTest
parent, parent,
null null
); );
final String origString = objectMapper.writeValueAsString(subTask); final String origString = jsonMapper.writeValueAsString(subTask);
final HadoopConverterTask.ConverterSubTask otherSub = objectMapper.readValue( final HadoopConverterTask.ConverterSubTask otherSub = jsonMapper.readValue(
origString, origString,
HadoopConverterTask.ConverterSubTask.class HadoopConverterTask.ConverterSubTask.class
); );
Assert.assertEquals(subTask, otherSub); Assert.assertEquals(subTask, otherSub);
Assert.assertEquals(origString, objectMapper.writeValueAsString(otherSub)); Assert.assertEquals(origString, jsonMapper.writeValueAsString(otherSub));
Assert.assertEquals(ImmutableList.of(DATA_SEGMENT), otherSub.getSegments()); Assert.assertEquals(ImmutableList.of(DATA_SEGMENT), otherSub.getSegments());
Assert.assertFalse(parent == otherSub.getParent()); Assert.assertFalse(parent == otherSub.getParent());
Assert.assertEquals(parent, otherSub.getParent()); Assert.assertEquals(parent, otherSub.getParent());

View File

@ -28,13 +28,16 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionClientFactory;
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.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
@ -63,8 +66,21 @@ public class IndexTaskTest
{ {
@Rule @Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder(); public TemporaryFolder temporaryFolder = new TemporaryFolder();
private final IndexSpec indexSpec = new IndexSpec(); private final IndexSpec indexSpec;
private final ObjectMapper jsonMapper = new DefaultObjectMapper(); private final ObjectMapper jsonMapper;
private IndexMerger indexMerger;
private IndexMaker indexMaker;
private IndexIO indexIO;
public IndexTaskTest()
{
indexSpec = new IndexSpec();
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
indexMerger = testUtils.getTestIndexMerger();
indexMaker = testUtils.getTestIndexMaker();
indexIO = testUtils.getTestIndexIO();
}
@Test @Test
public void testDeterminePartitions() throws Exception public void testDeterminePartitions() throws Exception
@ -128,7 +144,7 @@ public class IndexTaskTest
indexSpec indexSpec
) )
), ),
new DefaultObjectMapper(), jsonMapper,
null null
); );
@ -193,7 +209,7 @@ public class IndexTaskTest
), ),
null null
), ),
new DefaultObjectMapper(), jsonMapper,
null null
); );
@ -243,7 +259,8 @@ public class IndexTaskTest
segments.add(segment); segments.add(segment);
return segment; return segment;
} }
}, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder() }, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder(),
indexMerger, indexMaker, indexIO
) )
); );
@ -306,7 +323,7 @@ public class IndexTaskTest
), ),
null null
), ),
new DefaultObjectMapper(), jsonMapper,
null null
); );

View File

@ -20,6 +20,7 @@ package io.druid.indexing.common.task;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.hash.Hashing; import com.google.common.hash.Hashing;
import io.druid.indexing.common.TaskToolbox;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
@ -44,7 +45,7 @@ public class MergeTaskBaseTest
final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments, null) final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments, null)
{ {
@Override @Override
protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception protected File merge(TaskToolbox toolbox, Map<DataSegment, File> segments, File outDir) throws Exception
{ {
return null; return null;
} }

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common.task; package io.druid.indexing.common.task;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
@ -29,7 +28,7 @@ import io.druid.granularity.QueryGranularity;
import io.druid.guice.FirehoseModule; import io.druid.guice.FirehoseModule;
import io.druid.indexer.HadoopIOConfig; import io.druid.indexer.HadoopIOConfig;
import io.druid.indexer.HadoopIngestionSpec; import io.druid.indexer.HadoopIngestionSpec;
import io.druid.jackson.DefaultObjectMapper; import io.druid.indexing.common.TestUtils;
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;
@ -56,14 +55,19 @@ import java.io.IOException;
public class TaskSerdeTest public class TaskSerdeTest
{ {
private static final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
static {
jsonMapper = new DefaultObjectMapper();
jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper));
}
private final IndexSpec indexSpec = new IndexSpec(); private final IndexSpec indexSpec = new IndexSpec();
public TaskSerdeTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
}
@Test @Test
public void testIndexTaskSerde() throws Exception public void testIndexTaskSerde() throws Exception
{ {
@ -89,15 +93,10 @@ public class TaskSerdeTest
null null
); );
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
InjectableValues inject = new InjectableValues.Std()
.addValue(ObjectMapper.class, jsonMapper);
final String json = jsonMapper.writeValueAsString(task); final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final IndexTask task2 = jsonMapper.reader(Task.class).with(inject).readValue(json); final IndexTask task2 = (IndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval()); Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval());
@ -138,12 +137,11 @@ public class TaskSerdeTest
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule); jsonMapper.registerModule(jacksonModule);
} }
InjectableValues inject = new InjectableValues.Std()
.addValue(ObjectMapper.class, jsonMapper);
final String json = jsonMapper.writeValueAsString(task); final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final IndexTask task2 = jsonMapper.reader(Task.class).with(inject).readValue(json); final IndexTask task2 = (IndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval()); Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval());
@ -554,9 +552,7 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task); final String json = jsonMapper.writeValueAsString(task);
InjectableValues inject = new InjectableValues.Std() final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
.addValue(ObjectMapper.class, jsonMapper);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.reader(Task.class).with(inject).readValue(json);
Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals("foo", task.getDataSource());

View File

@ -48,18 +48,20 @@ import io.druid.granularity.QueryGranularity;
import io.druid.guice.GuiceInjectors; import io.druid.guice.GuiceInjectors;
import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.LocalTaskActionClientFactory;
import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.actions.TaskActionToolbox;
import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.HeapMemoryTaskStorage;
import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.TaskLockbox;
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.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.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger; import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexSchema;
@ -102,6 +104,18 @@ import java.util.Set;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class IngestSegmentFirehoseFactoryTest public class IngestSegmentFirehoseFactoryTest
{ {
private static final ObjectMapper MAPPER;
private static final IndexMerger INDEX_MERGER;
private static final IndexMaker INDEX_MAKER;
private static final IndexIO INDEX_IO;
static {
TestUtils testUtils = new TestUtils();
MAPPER = setupInjectablesInObjectMapper(testUtils.getTestObjectMapper());
INDEX_MERGER = testUtils.getTestIndexMerger();
INDEX_MAKER = testUtils.getTestIndexMaker();
INDEX_IO = testUtils.getTestIndexIO();
}
@Parameterized.Parameters(name = "{1}") @Parameterized.Parameters(name = "{1}")
public static Collection<Object[]> constructorFeeder() throws IOException public static Collection<Object[]> constructorFeeder() throws IOException
@ -136,7 +150,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, null, indexSpec); INDEX_MERGER.persist(index, persistDir, null, 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)
@ -179,7 +193,7 @@ public class IngestSegmentFirehoseFactoryTest
ts, ts,
new TaskActionToolbox(tl, mdc, newMockEmitter()) new TaskActionToolbox(tl, mdc, newMockEmitter())
); );
final ObjectMapper objectMapper = newObjectMapper();
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory( final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null), new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null),
tac, tac,
@ -244,10 +258,13 @@ public class IngestSegmentFirehoseFactoryTest
{ {
return Lists.newArrayList(); return Lists.newArrayList();
} }
}, objectMapper }, MAPPER
) )
), ),
objectMapper MAPPER,
INDEX_MERGER,
INDEX_MAKER,
INDEX_IO
); );
Collection<Object[]> values = new LinkedList<>(); Collection<Object[]> values = new LinkedList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList( for (InputRowParser parser : Arrays.<InputRowParser>asList(
@ -285,7 +302,8 @@ public class IngestSegmentFirehoseFactoryTest
binder.bind(TaskToolboxFactory.class).toInstance(taskToolboxFactory); binder.bind(TaskToolboxFactory.class).toInstance(taskToolboxFactory);
} }
} }
) ),
INDEX_IO
), ),
String.format( String.format(
"DimNames[%s]MetricNames[%s]ParserDimNames[%s]", "DimNames[%s]MetricNames[%s]ParserDimNames[%s]",
@ -302,9 +320,8 @@ public class IngestSegmentFirehoseFactoryTest
return values; return values;
} }
public static ObjectMapper newObjectMapper() public static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper)
{ {
final ObjectMapper objectMapper = new DefaultObjectMapper();
objectMapper.registerModule( objectMapper.registerModule(
new SimpleModule("testModule").registerSubtypes(LocalLoadSpec.class) new SimpleModule("testModule").registerSubtypes(LocalLoadSpec.class)
); );

View File

@ -43,6 +43,7 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClient;
@ -52,6 +53,8 @@ import io.druid.indexing.common.task.Task;
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.query.filter.NoopDimFilter; import io.druid.query.filter.NoopDimFilter;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger; import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; import io.druid.segment.IndexSpec;
import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexSchema;
@ -104,6 +107,19 @@ public class IngestSegmentFirehoseFactoryTimelineTest
private final int expectedCount; private final int expectedCount;
private final long expectedSum; private final long expectedSum;
private static final ObjectMapper MAPPER;
private static final IndexMerger INDEX_MERGER;
private static final IndexMaker INDEX_MAKER;
private static final IndexIO INDEX_IO;
static {
TestUtils testUtils = new TestUtils();
MAPPER = IngestSegmentFirehoseFactoryTest.setupInjectablesInObjectMapper(testUtils.getTestObjectMapper());
INDEX_MERGER = testUtils.getTestIndexMerger();
INDEX_MAKER = testUtils.getTestIndexMaker();
INDEX_IO = testUtils.getTestIndexIO();
}
public IngestSegmentFirehoseFactoryTimelineTest( public IngestSegmentFirehoseFactoryTimelineTest(
String name, String name,
IngestSegmentFirehoseFactory factory, IngestSegmentFirehoseFactory factory,
@ -212,7 +228,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
} }
try { try {
IndexMerger.persist(index, persistDir, null, new IndexSpec()); INDEX_MERGER.persist(index, persistDir, null, new IndexSpec());
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -262,7 +278,6 @@ public class IngestSegmentFirehoseFactoryTimelineTest
final List<Object[]> constructors = Lists.newArrayList(); final List<Object[]> constructors = Lists.newArrayList();
for (final TestCase testCase : testCases) { for (final TestCase testCase : testCases) {
final ObjectMapper objectMapper = IngestSegmentFirehoseFactoryTest.newObjectMapper();
final TaskActionClient taskActionClient = new TaskActionClient() final TaskActionClient taskActionClient = new TaskActionClient()
{ {
@Override @Override
@ -311,10 +326,13 @@ public class IngestSegmentFirehoseFactoryTimelineTest
{ {
return Lists.newArrayList(); return Lists.newArrayList();
} }
}, objectMapper }, MAPPER
) )
), ),
objectMapper MAPPER,
INDEX_MERGER,
INDEX_MAKER,
INDEX_IO
); );
final Injector injector = Guice.createInjector( final Injector injector = Guice.createInjector(
new Module() new Module()
@ -332,7 +350,8 @@ public class IngestSegmentFirehoseFactoryTimelineTest
new NoopDimFilter(), new NoopDimFilter(),
Arrays.asList(DIMENSIONS), Arrays.asList(DIMENSIONS),
Arrays.asList(METRICS), Arrays.asList(METRICS),
injector injector,
INDEX_IO
); );
constructors.add( constructors.add(

View File

@ -43,7 +43,6 @@ import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.worker.TaskAnnouncement; 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.server.initialization.IndexerZkConfig; import io.druid.server.initialization.IndexerZkConfig;
import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.initialization.ZkPathsConfig;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
@ -66,7 +65,6 @@ import java.util.concurrent.atomic.AtomicReference;
public class RemoteTaskRunnerTest public class RemoteTaskRunnerTest
{ {
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Joiner joiner = Joiner.on("/"); private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid"; private static final String basePath = "/test/druid";
private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath); private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath);
@ -74,6 +72,8 @@ public class RemoteTaskRunnerTest
private static final String statusPath = String.format("%s/indexer/status/worker", basePath); private static final String statusPath = String.format("%s/indexer/status/worker", basePath);
private static final int TIMEOUT_SECONDS = 5; private static final int TIMEOUT_SECONDS = 5;
private ObjectMapper jsonMapper;
private TestingCluster testingCluster; private TestingCluster testingCluster;
private CuratorFramework cf; private CuratorFramework cf;
private RemoteTaskRunner remoteTaskRunner; private RemoteTaskRunner remoteTaskRunner;
@ -85,6 +85,9 @@ public class RemoteTaskRunnerTest
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
{ {
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
testingCluster = new TestingCluster(1); testingCluster = new TestingCluster(1);
testingCluster.start(); testingCluster.start();

View File

@ -19,7 +19,6 @@ package io.druid.indexing.overlord;
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.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Function; import com.google.common.base.Function;
@ -79,6 +78,9 @@ import io.druid.query.QueryRunnerFactoryConglomerate;
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.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec; 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;
@ -127,6 +129,18 @@ import java.util.concurrent.TimeUnit;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class TaskLifecycleTest public class TaskLifecycleTest
{ {
private static final ObjectMapper MAPPER;
private static final IndexMerger INDEX_MERGER;
private static final IndexMaker INDEX_MAKER;
private static final IndexIO INDEX_IO;
static {
TestUtils testUtils = new TestUtils();
MAPPER = testUtils.getTestObjectMapper();
INDEX_MERGER = testUtils.getTestIndexMerger();
INDEX_IO = testUtils.getTestIndexIO();
INDEX_MAKER = testUtils.getTestIndexMaker();
}
@Parameterized.Parameters(name = "taskStorageType={0}") @Parameterized.Parameters(name = "taskStorageType={0}")
public static Collection<String[]> constructFeed() public static Collection<String[]> constructFeed()
@ -333,14 +347,15 @@ public class TaskLifecycleTest
announcedSinks = 0; announcedSinks = 0;
pushedSegments = 0; pushedSegments = 0;
tmpDir = temporaryFolder.newFolder(); tmpDir = temporaryFolder.newFolder();
TestUtils testUtils = new TestUtils();
mapper = testUtils.getTestObjectMapper();
tqc = new DefaultObjectMapper().readValue( tqc = mapper.readValue(
"{\"startDelay\":\"PT0S\", \"restartDelay\":\"PT1S\", \"storageSyncRate\":\"PT0.5S\"}", "{\"startDelay\":\"PT0S\", \"restartDelay\":\"PT1S\", \"storageSyncRate\":\"PT0.5S\"}",
TaskQueueConfig.class TaskQueueConfig.class
); );
indexSpec = new IndexSpec(); indexSpec = new IndexSpec();
mapper = new DefaultObjectMapper();
if (taskStorageType.equals("HeapMemoryTaskStorage")) { if (taskStorageType.equals("HeapMemoryTaskStorage")) {
ts = new HeapMemoryTaskStorage( ts = new HeapMemoryTaskStorage(
new TaskStorageConfig(null) new TaskStorageConfig(null)
@ -353,7 +368,6 @@ public class TaskLifecycleTest
new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"), new NamedType(MockExceptionalFirehoseFactory.class, "mockExcepFirehoseFactory"),
new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory") new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory")
); );
mapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, mapper));
testDerbyConnector.createTaskTables(); testDerbyConnector.createTaskTables();
testDerbyConnector.createSegmentTable(); testDerbyConnector.createSegmentTable();
ts = new MetadataTaskStorage( ts = new MetadataTaskStorage(
@ -471,7 +485,10 @@ public class TaskLifecycleTest
}, new DefaultObjectMapper() }, new DefaultObjectMapper()
) )
), ),
new DefaultObjectMapper() MAPPER,
INDEX_MERGER,
INDEX_MAKER,
INDEX_IO
); );
tr = new ThreadPoolTaskRunner(tb, null); tr = new ThreadPoolTaskRunner(tb, null);
tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter);
@ -505,7 +522,7 @@ public class TaskLifecycleTest
new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
), ),
TestUtils.MAPPER, mapper,
null null
); );
@ -563,7 +580,7 @@ public class TaskLifecycleTest
new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory()), new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory()),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
), ),
TestUtils.MAPPER, mapper,
null null
); );
@ -912,7 +929,7 @@ public class TaskLifecycleTest
new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)),
new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec)
), ),
TestUtils.MAPPER, mapper,
null null
); );

View File

@ -19,6 +19,7 @@ package io.druid.indexing.worker;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.RealtimeIndexTask;
import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.common.task.TaskResource;
@ -39,6 +40,14 @@ import java.io.File;
public class TaskAnnouncementTest public class TaskAnnouncementTest
{ {
private final ObjectMapper jsonMapper;
public TaskAnnouncementTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test @Test
public void testBackwardsCompatibleSerde() throws Exception public void testBackwardsCompatibleSerde() throws Exception
{ {
@ -68,7 +77,6 @@ public class TaskAnnouncementTest
final TaskStatus status = TaskStatus.running(task.getId()); final TaskStatus status = TaskStatus.running(task.getId());
final TaskAnnouncement announcement = TaskAnnouncement.create(task, status); final TaskAnnouncement announcement = TaskAnnouncement.create(task, status);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String statusJson = jsonMapper.writeValueAsString(status); final String statusJson = jsonMapper.writeValueAsString(status);
final String announcementJson = jsonMapper.writeValueAsString(announcement); final String announcementJson = jsonMapper.writeValueAsString(announcement);

View File

@ -34,7 +34,9 @@ import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.overlord.ThreadPoolTaskRunner;
import io.druid.indexing.worker.config.WorkerConfig; import io.druid.indexing.worker.config.WorkerConfig;
import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderConfig;
import io.druid.segment.loading.SegmentLoaderLocalCacheManager; import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.StorageLocationConfig; import io.druid.segment.loading.StorageLocationConfig;
@ -56,8 +58,6 @@ import java.util.List;
*/ */
public class WorkerTaskMonitorTest public class WorkerTaskMonitorTest
{ {
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Joiner joiner = Joiner.on("/"); private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid"; private static final String basePath = "/test/druid";
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath); private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
@ -71,6 +71,19 @@ public class WorkerTaskMonitorTest
private TestMergeTask task; private TestMergeTask task;
private Worker worker; private Worker worker;
private ObjectMapper jsonMapper;
private IndexMerger indexMerger;
private IndexMaker indexMaker;
private IndexIO indexIO;
public WorkerTaskMonitorTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
indexMerger = testUtils.getTestIndexMerger();
indexMaker = testUtils.getTestIndexMaker();
indexIO = testUtils.getTestIndexIO();
}
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
@ -143,7 +156,11 @@ public class WorkerTaskMonitorTest
} }
, jsonMapper , jsonMapper
) )
), jsonMapper ),
jsonMapper,
indexMerger,
indexMaker,
indexIO
), ),
null null
), ),

View File

@ -24,7 +24,6 @@ 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.base.Suppliers;
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;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -34,9 +33,7 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables; import com.google.common.io.Closeables;
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.Inject;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ConciseBitmapFactory; import com.metamx.collections.bitmap.ConciseBitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.bitmap.ImmutableBitmap;
@ -51,10 +48,6 @@ import com.metamx.common.io.smoosh.SmooshedWriter;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import io.druid.common.utils.SerializerUtils; import io.druid.common.utils.SerializerUtils;
import io.druid.guice.ConfigProvider;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.query.DruidProcessingConfig;
import io.druid.segment.column.Column; import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnConfig;
@ -113,64 +106,116 @@ public class IndexIO
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
private static final Map<Integer, IndexLoader> indexLoaders = private final Map<Integer, IndexLoader> indexLoaders;
ImmutableMap.<Integer, IndexLoader>builder()
.put(0, new LegacyIndexLoader())
.put(1, new LegacyIndexLoader())
.put(2, new LegacyIndexLoader())
.put(3, new LegacyIndexLoader())
.put(4, new LegacyIndexLoader())
.put(5, new LegacyIndexLoader())
.put(6, new LegacyIndexLoader())
.put(7, new LegacyIndexLoader())
.put(8, new LegacyIndexLoader())
.put(9, new V9IndexLoader())
.build();
private static final EmittingLogger log = new EmittingLogger(IndexIO.class); private static final EmittingLogger log = new EmittingLogger(IndexIO.class);
private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final SerializerUtils serializerUtils = new SerializerUtils();
private static final ObjectMapper mapper; private final ObjectMapper mapper;
private final DefaultIndexIOHandler defaultIndexIOHandler;
private final ColumnConfig columnConfig;
protected static final ColumnConfig columnConfig; @Inject
public IndexIO(ObjectMapper mapper, ColumnConfig columnConfig)
{
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
this.columnConfig = Preconditions.checkNotNull(columnConfig, "null ColumnConfig");
defaultIndexIOHandler = new DefaultIndexIOHandler(mapper);
indexLoaders = ImmutableMap.<Integer, IndexLoader>builder()
.put(0, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(1, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(2, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(3, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(4, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(5, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(6, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(7, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(8, new LegacyIndexLoader(defaultIndexIOHandler, columnConfig))
.put(9, new V9IndexLoader(columnConfig))
.build();
@Deprecated // specify bitmap type in IndexSpec instead
protected static final BitmapSerdeFactory CONFIGURED_BITMAP_SERDE_FACTORY;
static {
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.<Module>of(
new Module()
{
@Override
public void configure(Binder binder)
{
ConfigProvider.bind(
binder,
DruidProcessingConfig.class,
ImmutableMap.of("base_path", "druid.processing")
);
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
// this property is deprecated, use IndexSpec instead
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
}
}
)
);
mapper = injector.getInstance(ObjectMapper.class);
columnConfig = injector.getInstance(ColumnConfig.class);
CONFIGURED_BITMAP_SERDE_FACTORY = injector.getInstance(BitmapSerdeFactory.class);
} }
public static QueryableIndex loadIndex(File inDir) throws IOException public void validateTwoSegments(File dir1, File dir2) throws IOException
{
try (QueryableIndex queryableIndex1 = loadIndex(dir1)) {
try (QueryableIndex queryableIndex2 = loadIndex(dir2)) {
validateTwoSegments(
new QueryableIndexIndexableAdapter(queryableIndex1),
new QueryableIndexIndexableAdapter(queryableIndex2)
);
}
}
}
public void validateTwoSegments(final IndexableAdapter adapter1, final IndexableAdapter adapter2)
{
if (adapter1.getNumRows() != adapter2.getNumRows()) {
throw new SegmentValidationException(
"Row count mismatch. Expected [%d] found [%d]",
adapter1.getNumRows(),
adapter2.getNumRows()
);
}
{
final Set<String> dimNames1 = Sets.newHashSet(adapter1.getDimensionNames());
final Set<String> dimNames2 = Sets.newHashSet(adapter2.getDimensionNames());
if (!dimNames1.equals(dimNames2)) {
throw new SegmentValidationException(
"Dimension names differ. Expected [%s] found [%s]",
dimNames1,
dimNames2
);
}
final Set<String> metNames1 = Sets.newHashSet(adapter1.getMetricNames());
final Set<String> metNames2 = Sets.newHashSet(adapter2.getMetricNames());
if (!metNames1.equals(metNames2)) {
throw new SegmentValidationException("Metric names differ. Expected [%s] found [%s]", metNames1, metNames2);
}
}
final Iterator<Rowboat> it1 = adapter1.getRows().iterator();
final Iterator<Rowboat> it2 = adapter2.getRows().iterator();
long row = 0L;
while (it1.hasNext()) {
if (!it2.hasNext()) {
throw new SegmentValidationException("Unexpected end of second adapter");
}
final Rowboat rb1 = it1.next();
final Rowboat rb2 = it2.next();
++row;
if (rb1.getRowNum() != rb2.getRowNum()) {
throw new SegmentValidationException("Row number mismatch: [%d] vs [%d]", rb1.getRowNum(), rb2.getRowNum());
}
if (rb1.compareTo(rb2) != 0) {
try {
validateRowValues(rb1, adapter1, rb2, adapter2);
}
catch (SegmentValidationException ex) {
throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2);
}
}
}
if (it2.hasNext()) {
throw new SegmentValidationException("Unexpected end of first adapter");
}
if (row != adapter1.getNumRows()) {
throw new SegmentValidationException(
"Actual Row count mismatch. Expected [%d] found [%d]",
row,
adapter1.getNumRows()
);
}
}
public QueryableIndex loadIndex(File inDir) throws IOException
{ {
final int version = SegmentUtils.getVersionFromDir(inDir); final int version = SegmentUtils.getVersionFromDir(inDir);
final IndexLoader loader = indexLoaders.get(version); final IndexLoader loader = indexLoaders.get(version);
if (loader != null) { if (loader != null) {
return loader.load(inDir); return loader.load(inDir, mapper);
} else { } else {
throw new ISE("Unknown index version[%s]", version); throw new ISE("Unknown index version[%s]", version);
} }
@ -199,19 +244,19 @@ public class IndexIO
} }
} }
public static boolean convertSegment(File toConvert, File converted, IndexSpec indexSpec) throws IOException public boolean convertSegment(File toConvert, File converted, IndexSpec indexSpec) throws IOException
{ {
return convertSegment(toConvert, converted, indexSpec, false, true); return convertSegment(toConvert, converted, indexSpec, false, true);
} }
public static boolean convertSegment( public boolean convertSegment(
File toConvert, File toConvert,
File converted, File converted,
IndexSpec indexSpec, IndexSpec indexSpec,
boolean forceIfCurrent, boolean forceIfCurrent,
boolean validate boolean validate
) )
throws IOException throws IOException
{ {
final int version = SegmentUtils.getVersionFromDir(toConvert); final int version = SegmentUtils.getVersionFromDir(toConvert);
switch (version) { switch (version) {
@ -227,20 +272,20 @@ public class IndexIO
case 6: case 6:
case 7: case 7:
log.info("Old version, re-persisting."); log.info("Old version, re-persisting.");
IndexMerger.append( new IndexMerger(mapper, this).append(
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
converted, converted,
indexSpec indexSpec
); );
return true; return true;
case 8: case 8:
DefaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec); defaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec);
return true; return true;
default: default:
if (forceIfCurrent) { if (forceIfCurrent) {
IndexMerger.convert(toConvert, converted, indexSpec); new IndexMerger(mapper, this).convert(toConvert, converted, indexSpec);
if (validate) { if (validate) {
DefaultIndexIOHandler.validateTwoSegments(toConvert, converted); validateTwoSegments(toConvert, converted);
} }
return true; return true;
} else { } else {
@ -250,7 +295,12 @@ public class IndexIO
} }
} }
public static interface IndexIOHandler public DefaultIndexIOHandler getDefaultIndexIOHandler()
{
return defaultIndexIOHandler;
}
static interface IndexIOHandler
{ {
public MMappedIndex mapDir(File inDir) throws IOException; public MMappedIndex mapDir(File inDir) throws IOException;
} }
@ -370,6 +420,12 @@ public class IndexIO
public static class DefaultIndexIOHandler implements IndexIOHandler public static class DefaultIndexIOHandler implements IndexIOHandler
{ {
private static final Logger log = new Logger(DefaultIndexIOHandler.class); private static final Logger log = new Logger(DefaultIndexIOHandler.class);
private final ObjectMapper mapper;
public DefaultIndexIOHandler(ObjectMapper mapper)
{
this.mapper = mapper;
}
@Override @Override
public MMappedIndex mapDir(File inDir) throws IOException public MMappedIndex mapDir(File inDir) throws IOException
@ -473,78 +529,8 @@ public class IndexIO
return retVal; return retVal;
} }
public static void validateTwoSegments(File dir1, File dir2) throws IOException public void convertV8toV9(File v8Dir, File v9Dir, IndexSpec indexSpec)
{ throws IOException
try (QueryableIndex queryableIndex1 = loadIndex(dir1)) {
try (QueryableIndex queryableIndex2 = loadIndex(dir2)) {
validateTwoSegments(
new QueryableIndexIndexableAdapter(queryableIndex1),
new QueryableIndexIndexableAdapter(queryableIndex2)
);
}
}
}
public static void validateTwoSegments(final IndexableAdapter adapter1, final IndexableAdapter adapter2)
{
if (adapter1.getNumRows() != adapter2.getNumRows()) {
throw new SegmentValidationException(
"Row count mismatch. Expected [%d] found [%d]",
adapter1.getNumRows(),
adapter2.getNumRows()
);
}
{
final Set<String> dimNames1 = Sets.newHashSet(adapter1.getDimensionNames());
final Set<String> dimNames2 = Sets.newHashSet(adapter2.getDimensionNames());
if (!dimNames1.equals(dimNames2)) {
throw new SegmentValidationException(
"Dimension names differ. Expected [%s] found [%s]",
dimNames1,
dimNames2
);
}
final Set<String> metNames1 = Sets.newHashSet(adapter1.getMetricNames());
final Set<String> metNames2 = Sets.newHashSet(adapter2.getMetricNames());
if (!metNames1.equals(metNames2)) {
throw new SegmentValidationException("Metric names differ. Expected [%s] found [%s]", metNames1, metNames2);
}
}
final Iterator<Rowboat> it1 = adapter1.getRows().iterator();
final Iterator<Rowboat> it2 = adapter2.getRows().iterator();
long row = 0L;
while (it1.hasNext()) {
if (!it2.hasNext()) {
throw new SegmentValidationException("Unexpected end of second adapter");
}
final Rowboat rb1 = it1.next();
final Rowboat rb2 = it2.next();
++row;
if (rb1.getRowNum() != rb2.getRowNum()) {
throw new SegmentValidationException("Row number mismatch: [%d] vs [%d]", rb1.getRowNum(), rb2.getRowNum());
}
if (rb1.compareTo(rb2) != 0) {
try {
validateRowValues(rb1, adapter1, rb2, adapter2);
}
catch (SegmentValidationException ex) {
throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2);
}
}
}
if (it2.hasNext()) {
throw new SegmentValidationException("Unexpected end of first adapter");
}
if (row != adapter1.getNumRows()) {
throw new SegmentValidationException(
"Actual Row count mismatch. Expected [%d] found [%d]",
row,
adapter1.getNumRows()
);
}
}
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);
@ -888,15 +874,22 @@ public class IndexIO
static interface IndexLoader static interface IndexLoader
{ {
public QueryableIndex load(File inDir) throws IOException; public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException;
} }
static class LegacyIndexLoader implements IndexLoader static class LegacyIndexLoader implements IndexLoader
{ {
private static final IndexIOHandler legacyHandler = new DefaultIndexIOHandler(); private final IndexIOHandler legacyHandler;
private final ColumnConfig columnConfig;
LegacyIndexLoader(IndexIOHandler legacyHandler, ColumnConfig columnConfig)
{
this.legacyHandler = legacyHandler;
this.columnConfig = columnConfig;
}
@Override @Override
public QueryableIndex load(File inDir) throws IOException public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException
{ {
MMappedIndex index = legacyHandler.mapDir(inDir); MMappedIndex index = legacyHandler.mapDir(inDir);
@ -990,8 +983,15 @@ public class IndexIO
static class V9IndexLoader implements IndexLoader static class V9IndexLoader implements IndexLoader
{ {
private final ColumnConfig columnConfig;
V9IndexLoader(ColumnConfig columnConfig)
{
this.columnConfig = columnConfig;
}
@Override @Override
public QueryableIndex load(File inDir) throws IOException public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException
{ {
log.debug("Mapping v9 index[%s]", inDir); log.debug("Mapping v9 index[%s]", inDir);
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();

View File

@ -22,6 +22,7 @@ package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.base.Splitter; import com.google.common.base.Splitter;
import com.google.common.collect.FluentIterable; import com.google.common.collect.FluentIterable;
@ -35,8 +36,7 @@ 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.Injector; import com.google.inject.Inject;
import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.collections.bitmap.MutableBitmap;
@ -54,7 +54,6 @@ import com.metamx.common.logger.Logger;
import io.druid.collections.CombiningIterable; 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.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;
@ -110,14 +109,20 @@ public class IndexMaker
private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final SerializerUtils serializerUtils = new SerializerUtils();
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 final ObjectMapper mapper;
private final IndexIO indexIO;
static { @Inject
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.<Module>of()); public IndexMaker(
mapper = injector.getInstance(ObjectMapper.class); ObjectMapper mapper,
IndexIO indexIO
)
{
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
} }
public static File persist( public File persist(
final IncrementalIndex index, final IncrementalIndex index,
File outDir, File outDir,
final Map<String, Object> segmentMetadata, final Map<String, Object> segmentMetadata,
@ -137,7 +142,7 @@ public class IndexMaker
* *
* @throws java.io.IOException * @throws java.io.IOException
*/ */
public static File persist( public File persist(
final IncrementalIndex index, final IncrementalIndex index,
final Interval dataInterval, final Interval dataInterval,
File outDir, File outDir,
@ -150,7 +155,7 @@ public class IndexMaker
); );
} }
public static File persist( public File persist(
final IncrementalIndex index, final IncrementalIndex index,
final Interval dataInterval, final Interval dataInterval,
File outDir, File outDir,
@ -198,14 +203,14 @@ public class IndexMaker
); );
} }
public static File mergeQueryableIndex( public File mergeQueryableIndex(
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec
) throws IOException ) throws IOException
{ {
return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new LoggingProgressIndicator(outDir.toString())); return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new LoggingProgressIndicator(outDir.toString()));
} }
public static File mergeQueryableIndex( public File mergeQueryableIndex(
List<QueryableIndex> indexes, List<QueryableIndex> indexes,
final AggregatorFactory[] metricAggs, final AggregatorFactory[] metricAggs,
File outDir, File outDir,
@ -233,7 +238,7 @@ public class IndexMaker
); );
} }
public static File merge( public File merge(
List<IndexableAdapter> adapters, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec List<IndexableAdapter> adapters, final AggregatorFactory[] metricAggs, File outDir, final IndexSpec indexSpec
) throws IOException ) throws IOException
{ {
@ -242,7 +247,7 @@ public class IndexMaker
); );
} }
public static File merge( public File merge(
List<IndexableAdapter> adapters, List<IndexableAdapter> adapters,
final AggregatorFactory[] metricAggs, final AggregatorFactory[] metricAggs,
File outDir, File outDir,
@ -342,16 +347,16 @@ public class IndexMaker
} }
public static File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException
{ {
return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); return convert(inDir, outDir, indexSpec, new BaseProgressIndicator());
} }
public static File convert( public File convert(
final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress
) throws IOException ) throws IOException
{ {
try (QueryableIndex index = IndexIO.loadIndex(inDir)) { try (QueryableIndex index = indexIO.loadIndex(inDir)) {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
return makeIndexFiles( return makeIndexFiles(
ImmutableList.of(adapter), ImmutableList.of(adapter),
@ -374,7 +379,7 @@ public class IndexMaker
} }
} }
public static File append( public File append(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final File outDir, final File outDir,
final IndexSpec indexSpec final IndexSpec indexSpec
@ -383,7 +388,7 @@ public class IndexMaker
return append(adapters, outDir, new LoggingProgressIndicator(outDir.toString()), indexSpec); return append(adapters, outDir, new LoggingProgressIndicator(outDir.toString()), indexSpec);
} }
public static File append( public File append(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final File outDir, final File outDir,
final ProgressIndicator progress, final ProgressIndicator progress,
@ -459,7 +464,7 @@ public class IndexMaker
return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec); return makeIndexFiles(adapters, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec);
} }
private static File makeIndexFiles( private File makeIndexFiles(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final File outDir, final File outDir,
final ProgressIndicator progress, final ProgressIndicator progress,
@ -570,7 +575,7 @@ public class IndexMaker
return outDir; return outDir;
} }
private static void setupDimConversion( private void setupDimConversion(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final ProgressIndicator progress, final ProgressIndicator progress,
final List<String> mergedDimensions, final List<String> mergedDimensions,
@ -663,7 +668,7 @@ public class IndexMaker
progress.stopSection(section); progress.stopSection(section);
} }
private static Iterable<Rowboat> makeRowIterable( private Iterable<Rowboat> makeRowIterable(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final List<String> mergedDimensions, final List<String> mergedDimensions,
final List<String> mergedMetrics, final List<String> mergedMetrics,
@ -732,7 +737,7 @@ public class IndexMaker
return rowMergerFn.apply(boats); return rowMergerFn.apply(boats);
} }
private static int convertDims( private int convertDims(
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final ProgressIndicator progress, final ProgressIndicator progress,
final Iterable<Rowboat> theRows, final Iterable<Rowboat> theRows,
@ -775,7 +780,7 @@ public class IndexMaker
return rowCount; return rowCount;
} }
private static void makeTimeColumn( private void makeTimeColumn(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final ProgressIndicator progress, final ProgressIndicator progress,
final Iterable<Rowboat> theRows, final Iterable<Rowboat> theRows,
@ -812,7 +817,7 @@ public class IndexMaker
progress.stopSection(section); progress.stopSection(section);
} }
private static void makeDimColumns( private void makeDimColumns(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final ProgressIndicator progress, final ProgressIndicator progress,
@ -881,7 +886,7 @@ public class IndexMaker
} }
} }
private static void makeDimColumn( private void makeDimColumn(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final ProgressIndicator progress, final ProgressIndicator progress,
@ -1231,7 +1236,7 @@ public class IndexMaker
progress.stopSection(section); progress.stopSection(section);
} }
private static void makeMetricColumns( private void makeMetricColumns(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final ProgressIndicator progress, final ProgressIndicator progress,
final Iterable<Rowboat> theRows, final Iterable<Rowboat> theRows,
@ -1263,7 +1268,7 @@ public class IndexMaker
progress.stopSection(metSection); progress.stopSection(metSection);
} }
private static void makeMetricColumn( private void makeMetricColumn(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final ProgressIndicator progress, final ProgressIndicator progress,
final Iterable<Rowboat> theRows, final Iterable<Rowboat> theRows,
@ -1369,7 +1374,7 @@ public class IndexMaker
progress.stopSection(section); progress.stopSection(section);
} }
private static void makeIndexBinary( private void makeIndexBinary(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
final File outDir, final File outDir,
@ -1435,7 +1440,7 @@ public class IndexMaker
progress.stopSection(section); progress.stopSection(section);
} }
private static void makeMetadataBinary( private void makeMetadataBinary(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final ProgressIndicator progress, final ProgressIndicator progress,
final Map<String, Object> segmentMetadata final Map<String, Object> segmentMetadata
@ -1448,7 +1453,7 @@ public class IndexMaker
} }
} }
private static void writeColumn( private void writeColumn(
FileSmoosher v9Smoosher, FileSmoosher v9Smoosher,
ColumnPartSerde serde, ColumnPartSerde serde,
ColumnDescriptor.Builder builder, ColumnDescriptor.Builder builder,
@ -1471,7 +1476,7 @@ public class IndexMaker
channel.close(); channel.close();
} }
private static <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists) private <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
{ {
Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst()); Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst());

View File

@ -22,6 +22,7 @@ package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter; import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
@ -34,9 +35,7 @@ import com.google.common.io.ByteStreams;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.common.io.OutputSupplier; import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.google.inject.Binder; import com.google.inject.Inject;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.collections.bitmap.MutableBitmap;
@ -55,8 +54,6 @@ import io.druid.common.guava.FileOutputSupplier;
import io.druid.common.guava.GuavaUtils; import io.druid.common.guava.GuavaUtils;
import io.druid.common.utils.JodaUtils; import io.druid.common.utils.JodaUtils;
import io.druid.common.utils.SerializerUtils; import io.druid.common.utils.SerializerUtils;
import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnCapabilitiesImpl;
@ -109,26 +106,21 @@ public class IndexMerger
private static final int INVALID_ROW = -1; private static final int INVALID_ROW = -1;
private static final Splitter SPLITTER = Splitter.on(","); private static final Splitter SPLITTER = Splitter.on(",");
private static final ObjectMapper mapper; private final ObjectMapper mapper;
private final IndexIO indexIO;
@Inject
public IndexMerger(
ObjectMapper mapper,
IndexIO indexIO
)
{
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
static {
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.<Module>of(
new Module()
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.processing.bitmap", BitmapSerdeFactory.class);
}
}
)
);
mapper = injector.getInstance(ObjectMapper.class);
} }
public File persist(
public static File persist(
final IncrementalIndex index, final IncrementalIndex index,
File outDir, File outDir,
Map<String, Object> segmentMetadata, Map<String, Object> segmentMetadata,
@ -150,7 +142,7 @@ 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( public File persist(
final IncrementalIndex index, final IncrementalIndex index,
final Interval dataInterval, final Interval dataInterval,
File outDir, File outDir,
@ -161,7 +153,7 @@ public class IndexMerger
return persist(index, dataInterval, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator()); return persist(index, dataInterval, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator());
} }
public static File persist( public File persist(
final IncrementalIndex index, final IncrementalIndex index,
final Interval dataInterval, final Interval dataInterval,
File outDir, File outDir,
@ -209,14 +201,14 @@ public class IndexMerger
); );
} }
public static File mergeQueryableIndex( public File mergeQueryableIndex(
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec
) throws IOException ) throws IOException
{ {
return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
} }
public static File mergeQueryableIndex( public File mergeQueryableIndex(
List<QueryableIndex> indexes, List<QueryableIndex> indexes,
final AggregatorFactory[] metricAggs, final AggregatorFactory[] metricAggs,
File outDir, File outDir,
@ -244,7 +236,7 @@ public class IndexMerger
); );
} }
public static File merge( public File merge(
List<IndexableAdapter> indexes, List<IndexableAdapter> indexes,
final AggregatorFactory[] metricAggs, final AggregatorFactory[] metricAggs,
File outDir, File outDir,
@ -255,7 +247,7 @@ public class IndexMerger
return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator()); return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator());
} }
public static File merge( public File merge(
List<IndexableAdapter> indexes, List<IndexableAdapter> indexes,
final AggregatorFactory[] metricAggs, final AggregatorFactory[] metricAggs,
File outDir, File outDir,
@ -361,16 +353,16 @@ public class IndexMerger
} }
// Faster than IndexMaker // Faster than IndexMaker
public static File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException
{ {
return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); return convert(inDir, outDir, indexSpec, new BaseProgressIndicator());
} }
public static File convert( public File convert(
final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress
) throws IOException ) throws IOException
{ {
try (QueryableIndex index = IndexIO.loadIndex(inDir)) { try (QueryableIndex index = indexIO.loadIndex(inDir)) {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
return makeIndexFiles( return makeIndexFiles(
ImmutableList.of(adapter), ImmutableList.of(adapter),
@ -393,14 +385,14 @@ public class IndexMerger
} }
} }
public static File append( public File append(
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec
) throws IOException ) throws IOException
{ {
return append(indexes, outDir, indexSpec, new BaseProgressIndicator()); return append(indexes, outDir, indexSpec, new BaseProgressIndicator());
} }
public static File append( public File append(
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress
) throws IOException ) throws IOException
{ {
@ -473,7 +465,7 @@ public class IndexMerger
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec); return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec);
} }
private static File makeIndexFiles( private File makeIndexFiles(
final List<IndexableAdapter> indexes, final List<IndexableAdapter> indexes,
final File outDir, final File outDir,
final ProgressIndicator progress, final ProgressIndicator progress,
@ -954,13 +946,13 @@ public class IndexMerger
indexSpec.getBitmapSerdeFactory() indexSpec.getBitmapSerdeFactory()
); );
IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir, indexSpec); indexIO.getDefaultIndexIOHandler().convertV8toV9(v8OutDir, outDir, indexSpec);
FileUtils.deleteDirectory(v8OutDir); FileUtils.deleteDirectory(v8OutDir);
return outDir; return outDir;
} }
private static <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists) private <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
{ {
Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst()); Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst());
@ -973,7 +965,7 @@ public class IndexMerger
return Lists.newArrayList(retVal); return Lists.newArrayList(retVal);
} }
public static void createIndexDrdFile( public void createIndexDrdFile(
byte versionId, byte versionId,
File inDir, File inDir,
GenericIndexed<String> availableDimensions, GenericIndexed<String> availableDimensions,
@ -1292,7 +1284,7 @@ public class IndexMerger
return true; return true;
} }
private static void writeMetadataToFile(File metadataFile, Map<String, Object> metadata) throws IOException private void writeMetadataToFile(File metadataFile, Map<String, Object> metadata) throws IOException
{ {
try (FileOutputStream metadataFileOutputStream = new FileOutputStream(metadataFile); try (FileOutputStream metadataFileOutputStream = new FileOutputStream(metadataFile);
FileChannel metadataFilechannel = metadataFileOutputStream.getChannel() FileChannel metadataFilechannel = metadataFileOutputStream.getChannel()

View File

@ -27,6 +27,7 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.Arrays; import java.util.Arrays;
@ -98,7 +99,7 @@ public class IndexSpec
Preconditions.checkArgument(metricCompression == null || COMPRESSION_NAMES.contains(metricCompression), Preconditions.checkArgument(metricCompression == null || COMPRESSION_NAMES.contains(metricCompression),
"Unknown compression type[%s]", metricCompression); "Unknown compression type[%s]", metricCompression);
this.bitmapSerdeFactory = bitmapSerdeFactory != null ? bitmapSerdeFactory : IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY; this.bitmapSerdeFactory = bitmapSerdeFactory != null ? bitmapSerdeFactory : new ConciseBitmapSerdeFactory();
this.metricCompression = metricCompression; this.metricCompression = metricCompression;
this.dimensionCompression = dimensionCompression; this.dimensionCompression = dimensionCompression;
} }

View File

@ -59,6 +59,7 @@ 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;
import io.druid.segment.TestHelper;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.incremental.IndexSizeExceededException;
import io.druid.segment.incremental.OnheapIncrementalIndex; import io.druid.segment.incremental.OnheapIncrementalIndex;
@ -85,6 +86,8 @@ import java.util.Map;
public class AggregationTestHelper public class AggregationTestHelper
{ {
private final ObjectMapper mapper; private final ObjectMapper mapper;
private final IndexMerger indexMerger;
private final IndexIO indexIO;
private final GroupByQueryQueryToolChest toolChest; private final GroupByQueryQueryToolChest toolChest;
private final GroupByQueryRunnerFactory factory; private final GroupByQueryRunnerFactory factory;
@ -94,6 +97,8 @@ public class AggregationTestHelper
{ {
this.tempFolder = tempFoler; this.tempFolder = tempFoler;
mapper = new DefaultObjectMapper(); mapper = new DefaultObjectMapper();
indexIO = TestHelper.getTestIndexIO();
indexMerger = TestHelper.getTestIndexMerger();
for(Module mod : jsonModulesToRegister) { for(Module mod : jsonModulesToRegister) {
mapper.registerModule(mod); mapper.registerModule(mod);
@ -251,7 +256,7 @@ public class AggregationTestHelper
catch (IndexSizeExceededException ex) { catch (IndexSizeExceededException ex) {
File tmp = tempFolder.newFolder(); File tmp = tempFolder.newFolder();
toMerge.add(tmp); toMerge.add(tmp);
IndexMerger.persist(index, tmp, null, new IndexSpec()); indexMerger.persist(index, tmp, null, new IndexSpec());
index.close(); index.close();
index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, maxRowCount); index = new OnheapIncrementalIndex(minTimestamp, gran, metrics, deserializeComplexMetrics, maxRowCount);
} }
@ -260,19 +265,19 @@ public class AggregationTestHelper
if (toMerge.size() > 0) { if (toMerge.size() > 0) {
File tmp = tempFolder.newFolder(); File tmp = tempFolder.newFolder();
toMerge.add(tmp); toMerge.add(tmp);
IndexMerger.persist(index, tmp, null, new IndexSpec()); indexMerger.persist(index, tmp, null, new IndexSpec());
List<QueryableIndex> indexes = new ArrayList<>(toMerge.size()); List<QueryableIndex> indexes = new ArrayList<>(toMerge.size());
for (File file : toMerge) { for (File file : toMerge) {
indexes.add(IndexIO.loadIndex(file)); indexes.add(indexIO.loadIndex(file));
} }
IndexMerger.mergeQueryableIndex(indexes, metrics, outDir, new IndexSpec()); indexMerger.mergeQueryableIndex(indexes, metrics, outDir, new IndexSpec());
for (QueryableIndex qi : indexes) { for (QueryableIndex qi : indexes) {
qi.close(); qi.close();
} }
} else { } else {
IndexMerger.persist(index, outDir, null, new IndexSpec()); indexMerger.persist(index, outDir, null, new IndexSpec());
} }
} }
finally { finally {
@ -299,7 +304,7 @@ public class AggregationTestHelper
public QueryableIndexSegment apply(File segmentDir) public QueryableIndexSegment apply(File segmentDir)
{ {
try { try {
return new QueryableIndexSegment("", IndexIO.loadIndex(segmentDir)); return new QueryableIndexSegment("", indexIO.loadIndex(segmentDir));
} }
catch (IOException ex) { catch (IOException ex) {
throw Throwables.propagate(ex); throw Throwables.propagate(ex);

View File

@ -57,7 +57,7 @@ public class EmptyIndexTest
emptyIndex, emptyIndex,
new ConciseBitmapFactory() new ConciseBitmapFactory()
); );
IndexMerger.merge( TestHelper.getTestIndexMerger().merge(
Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter), Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter),
new AggregatorFactory[0], new AggregatorFactory[0],
tmpDir, tmpDir,
@ -65,7 +65,7 @@ public class EmptyIndexTest
new IndexSpec() new IndexSpec()
); );
QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir); QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO().loadIndex(tmpDir);
Assert.assertEquals("getDimensionNames", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); Assert.assertEquals("getDimensionNames", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions()));
Assert.assertEquals("getMetricNames", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); Assert.assertEquals("getMetricNames", 0, Iterables.size(emptyQueryableIndex.getColumnNames()));

View File

@ -337,7 +337,7 @@ public class IndexIOTest
{ {
Exception ex = null; Exception ex = null;
try { try {
IndexIO.DefaultIndexIOHandler.validateTwoSegments(adapter1, adapter2); TestHelper.getTestIndexIO().validateTwoSegments(adapter1, adapter2);
} }
catch (Exception e) { catch (Exception e) {
ex = e; ex = e;

View File

@ -63,6 +63,10 @@ public class IndexMakerParameterizedTest
@Rule @Rule
public final CloserRule closer = new CloserRule(false); public final CloserRule closer = new CloserRule(false);
private final static IndexMaker INDEX_MAKER = TestHelper.getTestIndexMaker();
private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
@Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}") @Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}")
public static Collection<Object[]> data() public static Collection<Object[]> data()
{ {
@ -129,8 +133,8 @@ public class IndexMakerParameterizedTest
final File tempDir = temporaryFolder.newFolder(); final File tempDir = temporaryFolder.newFolder();
QueryableIndex index = closer.closeLater( QueryableIndex index = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist, toPersist,
tempDir, tempDir,
null, null,
@ -181,8 +185,8 @@ public class IndexMakerParameterizedTest
final File mergedDir = temporaryFolder.newFolder(); final File mergedDir = temporaryFolder.newFolder();
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -196,8 +200,8 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(3, index1.getColumnNames().size()); Assert.assertEquals(3, index1.getColumnNames().size());
QueryableIndex index2 = closer.closeLater( QueryableIndex index2 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist2, toPersist2,
tempDir2, tempDir2,
null, null,
@ -211,8 +215,8 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(3, index2.getColumnNames().size()); Assert.assertEquals(3, index2.getColumnNames().size());
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.mergeQueryableIndex( INDEX_MAKER.mergeQueryableIndex(
Arrays.asList(index1, index2), Arrays.asList(index1, index2),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -265,8 +269,8 @@ public class IndexMakerParameterizedTest
); );
final QueryableIndex index1 = closer.closeLater( final QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tmpDir1, tmpDir1,
null, null,
@ -275,8 +279,8 @@ public class IndexMakerParameterizedTest
) )
); );
final QueryableIndex index2 = closer.closeLater( final QueryableIndex index2 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tmpDir2, tmpDir2,
null, null,
@ -285,8 +289,8 @@ public class IndexMakerParameterizedTest
) )
); );
final QueryableIndex merged = closer.closeLater( final QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3, indexSpec) INDEX_MAKER.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3, indexSpec)
) )
); );
@ -321,8 +325,8 @@ public class IndexMakerParameterizedTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -334,7 +338,7 @@ public class IndexMakerParameterizedTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -342,8 +346,8 @@ public class IndexMakerParameterizedTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.mergeQueryableIndex( INDEX_MAKER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -356,7 +360,7 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
@ -380,15 +384,15 @@ public class IndexMakerParameterizedTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.append( INDEX_MAKER.append(
ImmutableList.<IndexableAdapter>of(incrementalAdapter), tempDir1, indexSpec ImmutableList.<IndexableAdapter>of(incrementalAdapter), tempDir1, indexSpec
) )
) )
); );
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -396,8 +400,8 @@ public class IndexMakerParameterizedTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.mergeQueryableIndex( INDEX_MAKER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -410,7 +414,7 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
@ -433,8 +437,8 @@ public class IndexMakerParameterizedTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -446,7 +450,7 @@ public class IndexMakerParameterizedTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -461,8 +465,8 @@ public class IndexMakerParameterizedTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.mergeQueryableIndex( INDEX_MAKER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -475,7 +479,7 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, newSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, newSpec.getDimensionCompressionStrategy());
@ -499,8 +503,8 @@ public class IndexMakerParameterizedTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -512,7 +516,7 @@ public class IndexMakerParameterizedTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -520,8 +524,8 @@ public class IndexMakerParameterizedTest
QueryableIndex converted = closer.closeLater( QueryableIndex converted = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.convert( INDEX_MAKER.convert(
tempDir1, tempDir1,
convertDir, convertDir,
indexSpec indexSpec
@ -533,7 +537,7 @@ public class IndexMakerParameterizedTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions()));
Assert.assertEquals(3, converted.getColumnNames().size()); Assert.assertEquals(3, converted.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, convertDir); INDEX_IO.validateTwoSegments(tempDir1, convertDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy());
@ -557,8 +561,8 @@ public class IndexMakerParameterizedTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMaker.persist( INDEX_MAKER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -570,7 +574,7 @@ public class IndexMakerParameterizedTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -583,13 +587,13 @@ public class IndexMakerParameterizedTest
"lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4"
); );
QueryableIndex converted = closer.closeLater(IndexIO.loadIndex(IndexMaker.convert(tempDir1, convertDir, newSpec))); QueryableIndex converted = closer.closeLater(INDEX_IO.loadIndex(INDEX_MAKER.convert(tempDir1, convertDir, newSpec)));
Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions()));
Assert.assertEquals(3, converted.getColumnNames().size()); Assert.assertEquals(3, converted.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, convertDir); INDEX_IO.validateTwoSegments(tempDir1, convertDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(converted, newSpec.getDimensionCompressionStrategy()); assertDimCompression(converted, newSpec.getDimensionCompressionStrategy());

View File

@ -67,6 +67,11 @@ public class IndexMakerTest
"count" "count"
) )
}; };
private static final IndexMaker INDEX_MAKER = TestHelper.getTestIndexMaker();
private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO();
private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec(
new ConciseBitmapSerdeFactory(), new ConciseBitmapSerdeFactory(),
CompressedObjectStrategy.CompressionStrategy.LZ4, CompressedObjectStrategy.CompressionStrategy.LZ4,
@ -170,7 +175,7 @@ public class IndexMakerTest
} }
tmpDir = Files.createTempDir(); tmpDir = Files.createTempDir();
persistTmpDir = new File(tmpDir, "persistDir"); persistTmpDir = new File(tmpDir, "persistDir");
IndexMerger.persist(toPersist, persistTmpDir, null, INDEX_SPEC); INDEX_MERGER.persist(toPersist, persistTmpDir, null, INDEX_SPEC);
} }
@After @After
@ -187,7 +192,7 @@ public class IndexMakerTest
try { try {
outDir = Files.createTempDir(); outDir = Files.createTempDir();
Map<String, Object> segmentMetadata = ImmutableMap.<String, Object>of("key", "value"); Map<String, Object> segmentMetadata = ImmutableMap.<String, Object>of("key", "value");
index = IndexIO.loadIndex(IndexMaker.persist(toPersist, outDir, segmentMetadata, INDEX_SPEC)); index = INDEX_IO.loadIndex(INDEX_MAKER.persist(toPersist, outDir, segmentMetadata, INDEX_SPEC));
Assert.assertEquals(segmentMetadata, index.getMetaData()); Assert.assertEquals(segmentMetadata, index.getMetaData());
} }
@ -208,7 +213,7 @@ public class IndexMakerTest
{ {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater( closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
persistTmpDir persistTmpDir
) )
) )
@ -219,23 +224,23 @@ public class IndexMakerTest
private File reprocessAndValidate(File inDir, File tmpDir) throws IOException private File reprocessAndValidate(File inDir, File tmpDir) throws IOException
{ {
final File outDir = IndexMaker.convert( final File outDir = INDEX_MAKER.convert(
inDir, inDir,
tmpDir, tmpDir,
INDEX_SPEC INDEX_SPEC
); );
IndexIO.DefaultIndexIOHandler.validateTwoSegments(persistTmpDir, outDir); INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir; return outDir;
} }
private File appendAndValidate(File inDir, File tmpDir) throws IOException private File appendAndValidate(File inDir, File tmpDir) throws IOException
{ {
final File outDir = IndexMerger.append( final File outDir = INDEX_MERGER.append(
ImmutableList.<IndexableAdapter>of(new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(inDir)))), ImmutableList.<IndexableAdapter>of(new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(inDir)))),
tmpDir, tmpDir,
INDEX_SPEC INDEX_SPEC
); );
IndexIO.DefaultIndexIOHandler.validateTwoSegments(persistTmpDir, outDir); INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir; return outDir;
} }
@ -244,7 +249,7 @@ public class IndexMakerTest
{ {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater( closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
persistTmpDir persistTmpDir
) )
) )
@ -254,12 +259,12 @@ public class IndexMakerTest
reprocessAndValidate(persistTmpDir, tmpDir1); reprocessAndValidate(persistTmpDir, tmpDir1);
final File tmpDir2 = new File(tmpDir, "reprocessed2"); final File tmpDir2 = new File(tmpDir, "reprocessed2");
final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(tmpDir1))); final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1)));
Assert.assertEquals(events.size(), adapter2.getNumRows()); Assert.assertEquals(events.size(), adapter2.getNumRows());
reprocessAndValidate(tmpDir1, tmpDir2); reprocessAndValidate(tmpDir1, tmpDir2);
final File tmpDir3 = new File(tmpDir, "reprocessed3"); final File tmpDir3 = new File(tmpDir, "reprocessed3");
final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(tmpDir2))); final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2)));
Assert.assertEquals(events.size(), adapter3.getNumRows()); Assert.assertEquals(events.size(), adapter3.getNumRows());
reprocessAndValidate(tmpDir2, tmpDir3); reprocessAndValidate(tmpDir2, tmpDir3);
} }
@ -269,7 +274,7 @@ public class IndexMakerTest
{ {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater( closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
persistTmpDir persistTmpDir
) )
) )
@ -283,7 +288,7 @@ public class IndexMakerTest
{ {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater( closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
persistTmpDir persistTmpDir
) )
) )
@ -293,12 +298,12 @@ public class IndexMakerTest
appendAndValidate(persistTmpDir, tmpDir1); appendAndValidate(persistTmpDir, tmpDir1);
final File tmpDir2 = new File(tmpDir, "reprocessed2"); final File tmpDir2 = new File(tmpDir, "reprocessed2");
final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(tmpDir1))); final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1)));
Assert.assertEquals(events.size(), adapter2.getNumRows()); Assert.assertEquals(events.size(), adapter2.getNumRows());
appendAndValidate(tmpDir1, tmpDir2); appendAndValidate(tmpDir1, tmpDir2);
final File tmpDir3 = new File(tmpDir, "reprocessed3"); final File tmpDir3 = new File(tmpDir, "reprocessed3");
final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(tmpDir2))); final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2)));
Assert.assertEquals(events.size(), adapter3.getNumRows()); Assert.assertEquals(events.size(), adapter3.getNumRows());
appendAndValidate(tmpDir2, tmpDir3); appendAndValidate(tmpDir2, tmpDir3);
} }

View File

@ -62,6 +62,9 @@ public class IndexMergerTest
@Rule @Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder(); public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
@Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}") @Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}")
public static Collection<Object[]> data() public static Collection<Object[]> data()
{ {
@ -130,8 +133,8 @@ public class IndexMergerTest
final File tempDir = temporaryFolder.newFolder(); final File tempDir = temporaryFolder.newFolder();
QueryableIndex index = closer.closeLater( QueryableIndex index = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist, toPersist,
tempDir, tempDir,
null, null,
@ -159,8 +162,8 @@ public class IndexMergerTest
final File tempDir = temporaryFolder.newFolder(); final File tempDir = temporaryFolder.newFolder();
QueryableIndex index = closer.closeLater( QueryableIndex index = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist, toPersist,
tempDir, tempDir,
segmentMetadata, segmentMetadata,
@ -213,8 +216,8 @@ public class IndexMergerTest
final File mergedDir = temporaryFolder.newFolder(); final File mergedDir = temporaryFolder.newFolder();
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -228,8 +231,8 @@ public class IndexMergerTest
Assert.assertEquals(3, index1.getColumnNames().size()); Assert.assertEquals(3, index1.getColumnNames().size());
QueryableIndex index2 = closer.closeLater( QueryableIndex index2 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist2, toPersist2,
tempDir2, tempDir2,
null, null,
@ -243,8 +246,8 @@ public class IndexMergerTest
Assert.assertEquals(3, index2.getColumnNames().size()); Assert.assertEquals(3, index2.getColumnNames().size());
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(index1, index2), Arrays.asList(index1, index2),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -297,8 +300,8 @@ public class IndexMergerTest
); );
final QueryableIndex index1 = closer.closeLater( final QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tmpDir1, tmpDir1,
null, null,
@ -307,8 +310,8 @@ public class IndexMergerTest
) )
); );
final QueryableIndex index2 = closer.closeLater( final QueryableIndex index2 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tmpDir2, tmpDir2,
null, null,
@ -317,8 +320,8 @@ public class IndexMergerTest
) )
); );
final QueryableIndex merged = closer.closeLater( final QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(index1, index2), Arrays.asList(index1, index2),
new AggregatorFactory[]{}, new AggregatorFactory[]{},
tmpDir3, tmpDir3,
@ -358,8 +361,8 @@ public class IndexMergerTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -371,7 +374,7 @@ public class IndexMergerTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -379,8 +382,8 @@ public class IndexMergerTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -393,7 +396,7 @@ public class IndexMergerTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
@ -417,15 +420,15 @@ public class IndexMergerTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.append( INDEX_MERGER.append(
ImmutableList.<IndexableAdapter>of(incrementalAdapter), tempDir1, indexSpec ImmutableList.<IndexableAdapter>of(incrementalAdapter), tempDir1, indexSpec
) )
) )
); );
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -433,8 +436,8 @@ public class IndexMergerTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -447,7 +450,7 @@ public class IndexMergerTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy());
@ -470,8 +473,8 @@ public class IndexMergerTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -483,7 +486,7 @@ public class IndexMergerTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -498,8 +501,8 @@ public class IndexMergerTest
QueryableIndex merged = closer.closeLater( QueryableIndex merged = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
ImmutableList.of(index1), ImmutableList.of(index1),
new AggregatorFactory[]{new CountAggregatorFactory("count")}, new AggregatorFactory[]{new CountAggregatorFactory("count")},
mergedDir, mergedDir,
@ -512,7 +515,7 @@ public class IndexMergerTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions()));
Assert.assertEquals(3, merged.getColumnNames().size()); Assert.assertEquals(3, merged.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, mergedDir); INDEX_IO.validateTwoSegments(tempDir1, mergedDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(merged, newSpec.getDimensionCompressionStrategy()); assertDimCompression(merged, newSpec.getDimensionCompressionStrategy());
@ -545,12 +548,12 @@ public class IndexMergerTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1, null, indexSpec)) INDEX_IO.loadIndex(INDEX_MERGER.persist(toPersist1, tempDir1, null, indexSpec))
); );
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -558,8 +561,8 @@ public class IndexMergerTest
QueryableIndex converted = closer.closeLater( QueryableIndex converted = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.convert( INDEX_MERGER.convert(
tempDir1, tempDir1,
convertDir, convertDir,
indexSpec indexSpec
@ -571,7 +574,7 @@ public class IndexMergerTest
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions()));
Assert.assertEquals(4, converted.getColumnNames().size()); Assert.assertEquals(4, converted.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, convertDir); INDEX_IO.validateTwoSegments(tempDir1, convertDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy());
@ -603,8 +606,8 @@ public class IndexMergerTest
); );
QueryableIndex index1 = closer.closeLater( QueryableIndex index1 = closer.closeLater(
IndexIO.loadIndex( INDEX_IO.loadIndex(
IndexMerger.persist( INDEX_MERGER.persist(
toPersist1, toPersist1,
tempDir1, tempDir1,
null, null,
@ -616,7 +619,7 @@ public class IndexMergerTest
final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(incrementalAdapter, queryableAdapter); INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter);
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()));
@ -629,13 +632,13 @@ public class IndexMergerTest
"lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4"
); );
QueryableIndex converted = closer.closeLater(IndexIO.loadIndex(IndexMerger.convert(tempDir1, convertDir, newSpec))); QueryableIndex converted = closer.closeLater(INDEX_IO.loadIndex(INDEX_MERGER.convert(tempDir1, convertDir, newSpec)));
Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength());
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions()));
Assert.assertEquals(4, converted.getColumnNames().size()); Assert.assertEquals(4, converted.getColumnNames().size());
IndexIO.DefaultIndexIOHandler.validateTwoSegments(tempDir1, convertDir); INDEX_IO.validateTwoSegments(tempDir1, convertDir);
assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy());
assertDimCompression(converted, newSpec.getDimensionCompressionStrategy()); assertDimCompression(converted, newSpec.getDimensionCompressionStrategy());

View File

@ -29,13 +29,6 @@ import org.junit.Test;
public class IndexSpecTest 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 @Test
public void testSerde() throws Exception public void testSerde() throws Exception
{ {
@ -67,7 +60,6 @@ public class IndexSpecTest
public void testDefaults() throws Exception public void testDefaults() throws Exception
{ {
final IndexSpec spec = new IndexSpec(); 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.getDimensionCompressionStrategy());
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompressionStrategy()); Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompressionStrategy());
} }

View File

@ -86,6 +86,9 @@ public class SchemalessIndex
private static final Map<Integer, Map<Integer, QueryableIndex>> mergedIndexes = Maps.newHashMap(); private static final Map<Integer, Map<Integer, QueryableIndex>> mergedIndexes = Maps.newHashMap();
private static final List<QueryableIndex> rowPersistedIndexes = Lists.newArrayList(); private static final List<QueryableIndex> rowPersistedIndexes = Lists.newArrayList();
private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO();
private static IncrementalIndex index = null; private static IncrementalIndex index = null;
private static QueryableIndex mergedIndex = null; private static QueryableIndex mergedIndex = null;
@ -188,12 +191,12 @@ public class SchemalessIndex
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMerger.persist(top, topFile, null, indexSpec); INDEX_MERGER.persist(top, topFile, null, indexSpec);
IndexMerger.persist(bottom, bottomFile, null, indexSpec); INDEX_MERGER.persist(bottom, bottomFile, null, indexSpec);
mergedIndex = io.druid.segment.IndexIO.loadIndex( mergedIndex = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)),
METRIC_AGGS, METRIC_AGGS,
mergedFile, mergedFile,
indexSpec indexSpec
@ -236,8 +239,8 @@ public class SchemalessIndex
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
QueryableIndex index = IndexIO.loadIndex( QueryableIndex index = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)),
METRIC_AGGS, METRIC_AGGS,
mergedFile, mergedFile,
@ -276,8 +279,8 @@ public class SchemalessIndex
indexesToMerge.add(rowPersistedIndexes.get(indexes[i])); indexesToMerge.add(rowPersistedIndexes.get(indexes[i]));
} }
QueryableIndex index = IndexIO.loadIndex( QueryableIndex index = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile, indexSpec) INDEX_MERGER.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile, indexSpec)
); );
return index; return index;
@ -358,8 +361,8 @@ public class SchemalessIndex
tmpFile.mkdirs(); tmpFile.mkdirs();
tmpFile.deleteOnExit(); tmpFile.deleteOnExit();
IndexMerger.persist(rowIndex, tmpFile, null, indexSpec); INDEX_MERGER.persist(rowIndex, tmpFile, null, indexSpec);
rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile)); rowPersistedIndexes.add(INDEX_IO.loadIndex(tmpFile));
} }
} }
catch (IOException e) { catch (IOException e) {
@ -418,7 +421,7 @@ public class SchemalessIndex
theFile.mkdirs(); theFile.mkdirs();
theFile.deleteOnExit(); theFile.deleteOnExit();
filesToMap.add(theFile); filesToMap.add(theFile);
IndexMerger.persist(index, theFile, null, indexSpec); INDEX_MERGER.persist(index, theFile, null, indexSpec);
} }
return filesToMap; return filesToMap;
@ -469,7 +472,7 @@ public class SchemalessIndex
{ {
try { try {
return new RowboatFilteringIndexAdapter( return new RowboatFilteringIndexAdapter(
new QueryableIndexIndexableAdapter(IndexIO.loadIndex(chunk.getObject())), new QueryableIndexIndexableAdapter(INDEX_IO.loadIndex(chunk.getObject())),
new Predicate<Rowboat>() new Predicate<Rowboat>()
{ {
@Override @Override
@ -492,7 +495,7 @@ public class SchemalessIndex
) )
); );
return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile, indexSpec)); return INDEX_IO.loadIndex(INDEX_MERGER.append(adapters, mergedFile, indexSpec));
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -510,8 +513,8 @@ public class SchemalessIndex
List<File> filesToMap = makeFilesToMap(tmpFile, files); List<File> filesToMap = makeFilesToMap(tmpFile, files);
return IndexIO.loadIndex( return INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Lists.newArrayList( Lists.newArrayList(
Iterables.transform( Iterables.transform(
filesToMap, filesToMap,
@ -521,7 +524,7 @@ public class SchemalessIndex
public QueryableIndex apply(@Nullable File input) public QueryableIndex apply(@Nullable File input)
{ {
try { try {
return IndexIO.loadIndex(input); return INDEX_IO.loadIndex(input);
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -17,11 +17,14 @@
package io.druid.segment; package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
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 io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Result; import io.druid.query.Result;
import io.druid.segment.column.ColumnConfig;
import org.junit.Assert; import org.junit.Assert;
import java.util.Iterator; import java.util.Iterator;
@ -30,6 +33,42 @@ import java.util.Iterator;
*/ */
public class TestHelper public class TestHelper
{ {
private static final IndexMerger INDEX_MERGER;
private static final IndexMaker INDEX_MAKER;
private static final IndexIO INDEX_IO;
static {
ObjectMapper jsonMapper = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
jsonMapper,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
INDEX_MERGER = new IndexMerger(jsonMapper, INDEX_IO);
INDEX_MAKER = new IndexMaker(jsonMapper, INDEX_IO);
}
public static IndexMerger getTestIndexMerger()
{
return INDEX_MERGER;
}
public static IndexMaker getTestIndexMaker()
{
return INDEX_MAKER;
}
public static IndexIO getTestIndexIO()
{
return INDEX_IO;
}
public static <T> void assertExpectedResults(Iterable<Result<T>> expectedResults, Sequence<Result<T>> results) public static <T> void assertExpectedResults(Iterable<Result<T>> expectedResults, Sequence<Result<T>> results)
{ {
assertResults(expectedResults, Sequences.toList(results, Lists.<Result<T>>newArrayList()), ""); assertResults(expectedResults, Sequences.toList(results, Lists.<Result<T>>newArrayList()), "");

View File

@ -82,6 +82,9 @@ public class TestIndex
}; };
private static final IndexSpec indexSpec = new IndexSpec(); private static final IndexSpec indexSpec = new IndexSpec();
private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO();
static { static {
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
@ -142,12 +145,12 @@ public class TestIndex
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMerger.persist(top, DATA_INTERVAL, topFile, null, indexSpec); INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, null, indexSpec);
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile, null, indexSpec); INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, null, indexSpec);
mergedRealtime = IndexIO.loadIndex( mergedRealtime = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)),
METRIC_AGGS, METRIC_AGGS,
mergedFile, mergedFile,
indexSpec indexSpec
@ -252,8 +255,8 @@ public class TestIndex
someTmpFile.mkdirs(); someTmpFile.mkdirs();
someTmpFile.deleteOnExit(); someTmpFile.deleteOnExit();
IndexMerger.persist(index, someTmpFile, null, indexSpec); INDEX_MERGER.persist(index, someTmpFile, null, indexSpec);
return IndexIO.loadIndex(someTmpFile); return INDEX_IO.loadIndex(someTmpFile);
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -77,6 +77,9 @@ public class SpatialFilterBonusTest
new LongSumAggregatorFactory("val", "val") new LongSumAggregatorFactory("val", "val")
}; };
private static List<String> DIMS = Lists.newArrayList("dim", "dim.geo"); private static List<String> DIMS = Lists.newArrayList("dim", "dim.geo");
private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO();
private final Segment segment; private final Segment segment;
public SpatialFilterBonusTest(Segment segment) public SpatialFilterBonusTest(Segment segment)
@ -232,8 +235,8 @@ public class SpatialFilterBonusTest
tmpFile.mkdirs(); tmpFile.mkdirs();
tmpFile.deleteOnExit(); tmpFile.deleteOnExit();
IndexMerger.persist(theIndex, tmpFile, null, indexSpec); INDEX_MERGER.persist(theIndex, tmpFile, null, indexSpec);
return IndexIO.loadIndex(tmpFile); return INDEX_IO.loadIndex(tmpFile);
} }
private static QueryableIndex makeMergedQueryableIndex(final IndexSpec indexSpec) private static QueryableIndex makeMergedQueryableIndex(final IndexSpec indexSpec)
@ -412,13 +415,17 @@ public class SpatialFilterBonusTest
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMerger.persist(first, DATA_INTERVAL, firstFile, null, indexSpec); INDEX_MERGER.persist(first, DATA_INTERVAL, firstFile, null, indexSpec);
IndexMerger.persist(second, DATA_INTERVAL, secondFile, null, indexSpec); INDEX_MERGER.persist(second, DATA_INTERVAL, secondFile, null, indexSpec);
IndexMerger.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec); INDEX_MERGER.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec);
QueryableIndex mergedRealtime = IndexIO.loadIndex( QueryableIndex mergedRealtime = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), Arrays.asList(
INDEX_IO.loadIndex(firstFile),
INDEX_IO.loadIndex(secondFile),
INDEX_IO.loadIndex(thirdFile)
),
METRIC_AGGS, METRIC_AGGS,
mergedFile, mergedFile,
indexSpec indexSpec

View File

@ -70,6 +70,9 @@ import java.util.Random;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class SpatialFilterTest public class SpatialFilterTest
{ {
private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
public static final int NUM_POINTS = 5000; public static final int NUM_POINTS = 5000;
private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07");
@ -261,8 +264,8 @@ public class SpatialFilterTest
tmpFile.mkdirs(); tmpFile.mkdirs();
tmpFile.deleteOnExit(); tmpFile.deleteOnExit();
IndexMerger.persist(theIndex, tmpFile, null, indexSpec); INDEX_MERGER.persist(theIndex, tmpFile, null, indexSpec);
return IndexIO.loadIndex(tmpFile); return INDEX_IO.loadIndex(tmpFile);
} }
private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec)
@ -481,13 +484,13 @@ public class SpatialFilterTest
mergedFile.mkdirs(); mergedFile.mkdirs();
mergedFile.deleteOnExit(); mergedFile.deleteOnExit();
IndexMerger.persist(first, DATA_INTERVAL, firstFile, null, indexSpec); INDEX_MERGER.persist(first, DATA_INTERVAL, firstFile, null, indexSpec);
IndexMerger.persist(second, DATA_INTERVAL, secondFile, null, indexSpec); INDEX_MERGER.persist(second, DATA_INTERVAL, secondFile, null, indexSpec);
IndexMerger.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec); INDEX_MERGER.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec);
QueryableIndex mergedRealtime = IndexIO.loadIndex( QueryableIndex mergedRealtime = INDEX_IO.loadIndex(
IndexMerger.mergeQueryableIndex( INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), Arrays.asList(INDEX_IO.loadIndex(firstFile), INDEX_IO.loadIndex(secondFile), INDEX_IO.loadIndex(thirdFile)),
METRIC_AGGS, METRIC_AGGS,
mergedFile, mergedFile,
indexSpec indexSpec

View File

@ -17,6 +17,8 @@
package io.druid.segment.loading; package io.druid.segment.loading;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.segment.IndexIO; import io.druid.segment.IndexIO;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
@ -30,11 +32,19 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory
{ {
private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); private static final Logger log = new Logger(MMappedQueryableIndexFactory.class);
private final IndexIO indexIO;
@Inject
public MMappedQueryableIndexFactory(IndexIO indexIO)
{
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
}
@Override @Override
public QueryableIndex factorize(File parentDir) throws SegmentLoadingException public QueryableIndex factorize(File parentDir) throws SegmentLoadingException
{ {
try { try {
return IndexIO.loadIndex(parentDir); return indexIO.loadIndex(parentDir);
} }
catch (IOException e) { catch (IOException e) {
throw new SegmentLoadingException(e, "%s", e.getMessage()); throw new SegmentLoadingException(e, "%s", e.getMessage());

View File

@ -25,6 +25,9 @@ import com.metamx.emitter.service.ServiceEmitter;
import io.druid.common.guava.ThreadRenamingCallable; import io.druid.common.guava.ThreadRenamingCallable;
import io.druid.concurrent.Execs; import io.druid.concurrent.Execs;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireDepartmentMetrics;
@ -59,7 +62,10 @@ public class FlushingPlumber extends RealtimePlumber
ServiceEmitter emitter, ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate, QueryRunnerFactoryConglomerate conglomerate,
DataSegmentAnnouncer segmentAnnouncer, DataSegmentAnnouncer segmentAnnouncer,
ExecutorService queryExecutorService ExecutorService queryExecutorService,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
) )
{ {
super( super(
@ -72,7 +78,10 @@ public class FlushingPlumber extends RealtimePlumber
queryExecutorService, queryExecutorService,
null, null,
null, null,
null null,
indexMerger,
indexMaker,
indexIO
); );
this.flushDuration = flushDuration; this.flushDuration = flushDuration;

View File

@ -24,6 +24,9 @@ import com.google.common.base.Preconditions;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Processing;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireDepartmentMetrics;
@ -46,6 +49,9 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
private final QueryRunnerFactoryConglomerate conglomerate; private final QueryRunnerFactoryConglomerate conglomerate;
private final DataSegmentAnnouncer segmentAnnouncer; private final DataSegmentAnnouncer segmentAnnouncer;
private final ExecutorService queryExecutorService; private final ExecutorService queryExecutorService;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@JsonCreator @JsonCreator
public FlushingPlumberSchool( public FlushingPlumberSchool(
@ -53,7 +59,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
@JacksonInject ServiceEmitter emitter, @JacksonInject ServiceEmitter emitter,
@JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject QueryRunnerFactoryConglomerate conglomerate,
@JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject DataSegmentAnnouncer segmentAnnouncer,
@JacksonInject @Processing ExecutorService queryExecutorService @JacksonInject @Processing ExecutorService queryExecutorService,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexMaker indexMaker,
@JacksonInject IndexIO indexIO
) )
{ {
super( super(
@ -63,7 +72,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
segmentAnnouncer, segmentAnnouncer,
null, null,
null, null,
queryExecutorService queryExecutorService,
indexMerger,
indexMaker,
indexIO
); );
this.flushDuration = flushDuration == null ? defaultFlushDuration : flushDuration; this.flushDuration = flushDuration == null ? defaultFlushDuration : flushDuration;
@ -71,6 +83,9 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
this.conglomerate = conglomerate; this.conglomerate = conglomerate;
this.segmentAnnouncer = segmentAnnouncer; this.segmentAnnouncer = segmentAnnouncer;
this.queryExecutorService = queryExecutorService; this.queryExecutorService = queryExecutorService;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexMaker = Preconditions.checkNotNull(indexMaker, "Null IndexMaker");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
} }
@Override @Override
@ -90,7 +105,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
emitter, emitter,
conglomerate, conglomerate,
segmentAnnouncer, segmentAnnouncer,
queryExecutorService queryExecutorService,
indexMerger,
indexMaker,
indexIO
); );
} }

View File

@ -21,6 +21,7 @@ package io.druid.segment.realtime.plumber;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.base.Stopwatch; import com.google.common.base.Stopwatch;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
@ -129,6 +130,9 @@ public class RealtimePlumber implements Plumber
private volatile ExecutorService persistExecutor = null; private volatile ExecutorService persistExecutor = null;
private volatile ExecutorService mergeExecutor = null; private volatile ExecutorService mergeExecutor = null;
private volatile ScheduledExecutorService scheduledExecutor = null; private volatile ScheduledExecutorService scheduledExecutor = null;
private volatile IndexMerger indexMerger;
private volatile IndexMaker indexMaker;
private volatile IndexIO indexIO;
private static final String COMMIT_METADATA_KEY = "%commitMetadata%"; private static final String COMMIT_METADATA_KEY = "%commitMetadata%";
private static final String COMMIT_METADATA_TIMESTAMP_KEY = "%commitMetadataTimestamp%"; private static final String COMMIT_METADATA_TIMESTAMP_KEY = "%commitMetadataTimestamp%";
@ -143,7 +147,10 @@ public class RealtimePlumber implements Plumber
ExecutorService queryExecutorService, ExecutorService queryExecutorService,
DataSegmentPusher dataSegmentPusher, DataSegmentPusher dataSegmentPusher,
SegmentPublisher segmentPublisher, SegmentPublisher segmentPublisher,
FilteredServerView serverView FilteredServerView serverView,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
) )
{ {
this.schema = schema; this.schema = schema;
@ -157,6 +164,9 @@ public class RealtimePlumber implements Plumber
this.dataSegmentPusher = dataSegmentPusher; this.dataSegmentPusher = dataSegmentPusher;
this.segmentPublisher = segmentPublisher; this.segmentPublisher = segmentPublisher;
this.serverView = serverView; this.serverView = serverView;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexMaker = Preconditions.checkNotNull(indexMaker, "Null IndexMaker");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy());
} }
@ -500,14 +510,14 @@ public class RealtimePlumber implements Plumber
final File mergedFile; final File mergedFile;
if (config.isPersistInHeap()) { if (config.isPersistInHeap()) {
mergedFile = IndexMaker.mergeQueryableIndex( mergedFile = indexMaker.mergeQueryableIndex(
indexes, indexes,
schema.getAggregators(), schema.getAggregators(),
mergedTarget, mergedTarget,
config.getIndexSpec() config.getIndexSpec()
); );
} else { } else {
mergedFile = IndexMerger.mergeQueryableIndex( mergedFile = indexMerger.mergeQueryableIndex(
indexes, indexes,
schema.getAggregators(), schema.getAggregators(),
mergedTarget, mergedTarget,
@ -518,7 +528,7 @@ public class RealtimePlumber implements Plumber
metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime); metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime);
metrics.incrementMergeTimeMillis(mergeStopwatch.elapsed(TimeUnit.MILLISECONDS)); metrics.incrementMergeTimeMillis(mergeStopwatch.elapsed(TimeUnit.MILLISECONDS));
QueryableIndex index = IndexIO.loadIndex(mergedFile); QueryableIndex index = indexIO.loadIndex(mergedFile);
log.info("Pushing [%s] to deep storage", sink.getSegment().getIdentifier()); log.info("Pushing [%s] to deep storage", sink.getSegment().getIdentifier());
DataSegment segment = dataSegmentPusher.push( DataSegment segment = dataSegmentPusher.push(
@ -705,7 +715,7 @@ public class RealtimePlumber implements Plumber
} }
QueryableIndex queryableIndex = null; QueryableIndex queryableIndex = null;
try { try {
queryableIndex = IndexIO.loadIndex(segmentDir); queryableIndex = indexIO.loadIndex(segmentDir);
} }
catch (IOException e) { catch (IOException e) {
log.error(e, "Problem loading segmentDir from disk."); log.error(e, "Problem loading segmentDir from disk.");
@ -974,14 +984,14 @@ public class RealtimePlumber implements Plumber
final IndexSpec indexSpec = config.getIndexSpec(); 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())),
metaData, metaData,
indexSpec 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())),
metaData, metaData,
@ -992,7 +1002,7 @@ public class RealtimePlumber implements Plumber
indexToPersist.swapSegment( indexToPersist.swapSegment(
new QueryableIndexSegment( new QueryableIndexSegment(
indexToPersist.getSegment().getIdentifier(), indexToPersist.getSegment().getIdentifier(),
IndexIO.loadIndex(persistedFile) indexIO.loadIndex(persistedFile)
) )
); );
return numRows; return numRows;

View File

@ -24,6 +24,9 @@ import com.metamx.emitter.service.ServiceEmitter;
import io.druid.client.FilteredServerView; import io.druid.client.FilteredServerView;
import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Processing;
import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.IndexMerger;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.DataSegmentPusher;
@ -44,6 +47,9 @@ public class RealtimePlumberSchool implements PlumberSchool
private final SegmentPublisher segmentPublisher; private final SegmentPublisher segmentPublisher;
private final FilteredServerView serverView; private final FilteredServerView serverView;
private final ExecutorService queryExecutorService; private final ExecutorService queryExecutorService;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@JsonCreator @JsonCreator
public RealtimePlumberSchool( public RealtimePlumberSchool(
@ -53,7 +59,10 @@ public class RealtimePlumberSchool implements PlumberSchool
@JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject DataSegmentAnnouncer segmentAnnouncer,
@JacksonInject SegmentPublisher segmentPublisher, @JacksonInject SegmentPublisher segmentPublisher,
@JacksonInject FilteredServerView serverView, @JacksonInject FilteredServerView serverView,
@JacksonInject @Processing ExecutorService executorService @JacksonInject @Processing ExecutorService executorService,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexMaker indexMaker,
@JacksonInject IndexIO indexIO
) )
{ {
this.emitter = emitter; this.emitter = emitter;
@ -63,6 +72,9 @@ public class RealtimePlumberSchool implements PlumberSchool
this.segmentPublisher = segmentPublisher; this.segmentPublisher = segmentPublisher;
this.serverView = serverView; this.serverView = serverView;
this.queryExecutorService = executorService; this.queryExecutorService = executorService;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexMaker = Preconditions.checkNotNull(indexMaker, "Null IndexMaker");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
} }
@Override @Override
@ -84,7 +96,10 @@ public class RealtimePlumberSchool implements PlumberSchool
queryExecutorService, queryExecutorService,
dataSegmentPusher, dataSegmentPusher,
segmentPublisher, segmentPublisher,
serverView serverView,
indexMerger,
indexMaker,
indexIO
); );
} }

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.TestHelper;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -72,7 +73,7 @@ public class SegmentLoaderLocalCacheManagerTest
locations.add(locationConfig); locations.add(locationConfig);
manager = new SegmentLoaderLocalCacheManager( manager = new SegmentLoaderLocalCacheManager(
new MMappedQueryableIndexFactory(), new MMappedQueryableIndexFactory(TestHelper.getTestIndexIO()),
new SegmentLoaderConfig().withLocations(locations), new SegmentLoaderConfig().withLocations(locations),
jsonMapper jsonMapper
); );

View File

@ -28,6 +28,7 @@ import io.druid.granularity.QueryGranularity;
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.CountAggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.TestHelper;
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;
@ -78,7 +79,16 @@ public class FireDepartmentTest
new RealtimeIOConfig( new RealtimeIOConfig(
null, null,
new RealtimePlumberSchool( new RealtimePlumberSchool(
null, null, null, null, null, null, null null,
null,
null,
null,
null,
null,
null,
TestHelper.getTestIndexMerger(),
TestHelper.getTestIndexMaker(),
TestHelper.getTestIndexIO()
), ),
null null
), ),

View File

@ -36,6 +36,7 @@ import io.druid.segment.IndexSpec;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.QueryableIndexStorageAdapter;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;
import io.druid.segment.TestHelper;
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 org.junit.Assert; import org.junit.Assert;
@ -54,6 +55,9 @@ public class IngestSegmentFirehoseTest
@Rule @Rule
public final TemporaryFolder tempFolder = new TemporaryFolder(); public final TemporaryFolder tempFolder = new TemporaryFolder();
private IndexIO indexIO = TestHelper.getTestIndexIO();
private IndexMerger indexMerger = TestHelper.getTestIndexMerger();
@Test @Test
public void testSanity() throws Exception public void testSanity() throws Exception
{ {
@ -62,7 +66,7 @@ public class IngestSegmentFirehoseTest
QueryableIndex qi = null; QueryableIndex qi = null;
try { try {
qi = IndexIO.loadIndex(segmentDir); qi = indexIO.loadIndex(segmentDir);
StorageAdapter sa = new QueryableIndexStorageAdapter(qi); StorageAdapter sa = new QueryableIndexStorageAdapter(qi);
WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval());
IngestSegmentFirehose firehose = new IngestSegmentFirehose( IngestSegmentFirehose firehose = new IngestSegmentFirehose(
@ -121,7 +125,7 @@ public class IngestSegmentFirehoseTest
for (String line : rows) { for (String line : rows) {
index.add(parser.parse(line)); index.add(parser.parse(line));
} }
IndexMerger.persist(index, segmentDir, null, new IndexSpec()); indexMerger.persist(index, segmentDir, null, new IndexSpec());
} }
finally { finally {
if (index != null) { if (index != null) {

View File

@ -30,14 +30,11 @@ import com.metamx.common.Granularity;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import io.druid.client.FilteredServerView; import io.druid.client.FilteredServerView;
import io.druid.client.ServerView; import io.druid.client.ServerView;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.Committer; import io.druid.data.input.Committer;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.JSONParseSpec;
import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec; import io.druid.data.input.impl.TimestampSpec;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
@ -47,6 +44,7 @@ import io.druid.query.Query;
import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactory;
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.TestHelper;
import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -57,7 +55,6 @@ import io.druid.segment.realtime.SegmentPublisher;
import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -198,7 +195,10 @@ public class RealtimePlumberSchoolTest
announcer, announcer,
segmentPublisher, segmentPublisher,
serverView, serverView,
MoreExecutors.sameThreadExecutor() MoreExecutors.sameThreadExecutor(),
TestHelper.getTestIndexMerger(),
TestHelper.getTestIndexMaker(),
TestHelper.getTestIndexIO()
); );
metrics = new FireDepartmentMetrics(); metrics = new FireDepartmentMetrics();

View File

@ -38,11 +38,7 @@ import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.JobHelper; import io.druid.indexer.JobHelper;
import io.druid.indexer.Jobby; import io.druid.indexer.Jobby;
import io.druid.indexer.MetadataStorageUpdaterJobHandler; import io.druid.indexer.MetadataStorageUpdaterJobHandler;
import io.druid.indexer.hadoop.DatasourceIngestionSpec;
import io.druid.indexer.path.DatasourcePathSpec;
import io.druid.indexer.path.MetadataStoreBasedUsedSegmentLister; import io.druid.indexer.path.MetadataStoreBasedUsedSegmentLister;
import io.druid.indexer.path.MultiplePathSpec;
import io.druid.indexer.path.PathSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec; import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
@ -119,7 +115,7 @@ public class CliInternalHadoopIndexer extends GuiceRunnable
config = HadoopDruidIndexerConfig.fromSpec( config = HadoopDruidIndexerConfig.fromSpec(
HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed( HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(
config.getSchema(), config.getSchema(),
HadoopDruidIndexerConfig.jsonMapper, HadoopDruidIndexerConfig.JSON_MAPPER,
new MetadataStoreBasedUsedSegmentLister( new MetadataStoreBasedUsedSegmentLister(
injector.getInstance(IndexerMetadataStorageCoordinator.class) injector.getInstance(IndexerMetadataStorageCoordinator.class)
) )