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?|
|--------|-----------|---------------|-------|---------|
|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|
|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.LongWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.Reducer;
@ -132,7 +131,7 @@ public class DetermineHashedPartitionsJob implements Jobby
if (!Utils.exists(groupByJob, fileSystem, 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>>()
{
}
@ -156,7 +155,7 @@ public class DetermineHashedPartitionsJob implements Jobby
fileSystem = partitionInfoPath.getFileSystem(groupByJob.getConfiguration());
}
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>()
{
}
@ -178,7 +177,7 @@ public class DetermineHashedPartitionsJob implements Jobby
new HashBasedNumberedShardSpec(
i,
numberOfShards,
HadoopDruidIndexerConfig.jsonMapper
HadoopDruidIndexerConfig.JSON_MAPPER
),
shardCount++
)
@ -267,7 +266,7 @@ public class DetermineHashedPartitionsJob implements Jobby
}
hyperLogLogs.get(interval)
.add(
hashFunction.hashBytes(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey))
hashFunction.hashBytes(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey))
.asBytes()
);
}
@ -324,7 +323,7 @@ public class DetermineHashedPartitionsJob implements Jobby
);
try {
HadoopDruidIndexerConfig.jsonMapper.writerWithType(
HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(
new TypeReference<Long>()
{
}
@ -350,7 +349,7 @@ public class DetermineHashedPartitionsJob implements Jobby
);
try {
HadoopDruidIndexerConfig.jsonMapper.writerWithType(
HadoopDruidIndexerConfig.JSON_MAPPER.writerWithType(
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.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.InvalidJobConfException;
import org.apache.hadoop.mapreduce.Job;
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 Joiner tabJoiner = HadoopDruidIndexerConfig.tabJoiner;
private static final Splitter tabSplitter = HadoopDruidIndexerConfig.tabSplitter;
private static final Joiner TAB_JOINER = HadoopDruidIndexerConfig.TAB_JOINER;
private static final Splitter TAB_SPLITTER = HadoopDruidIndexerConfig.TAB_SPLITTER;
private final HadoopDruidIndexerConfig config;
@ -225,7 +224,7 @@ public class DeterminePartitionsJob implements Jobby
fileSystem = partitionInfoPath.getFileSystem(dimSelectionJob.getConfiguration());
}
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>>()
{
}
@ -275,7 +274,7 @@ public class DeterminePartitionsJob implements Jobby
inputRow
);
context.write(
new BytesWritable(HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(groupKey)),
new BytesWritable(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)),
NullWritable.get()
);
}
@ -317,7 +316,7 @@ public class DeterminePartitionsJob implements Jobby
BytesWritable key, NullWritable value, Context context
) 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 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:");
for (ShardSpec shardSpec : chosenShardSpecs) {
log.info(" %s", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(shardSpec));
log.info(" %s", HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(shardSpec));
}
try {
HadoopDruidIndexerConfig.jsonMapper
HadoopDruidIndexerConfig.JSON_MAPPER
.writerWithType(
new TypeReference<List<ShardSpec>>()
{
@ -883,12 +882,12 @@ public class DeterminePartitionsJob implements Jobby
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)
{
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 int numRows = Integer.parseInt(splits.next());
final String value = splits.next();
@ -906,8 +905,8 @@ public class DeterminePartitionsJob implements Jobby
{
context.write(
new SortableBytes(
groupKey, tabJoiner.join(dimValueCount.dim, dimValueCount.value).getBytes(
HadoopDruidIndexerConfig.javaNativeCharset
groupKey, TAB_JOINER.join(dimValueCount.dim, dimValueCount.value).getBytes(
HadoopDruidIndexerConfig.JAVA_NATIVE_CHARSET
)
).toBytesWritable(),
dimValueCount.toText()

View File

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

View File

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

View File

@ -19,7 +19,6 @@ package io.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
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.offheap.OffheapBufferPool;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMaker;
import io.druid.segment.LoggingProgressIndicator;
import io.druid.segment.ProgressIndicator;
import io.druid.segment.QueryableIndex;
@ -85,7 +82,7 @@ public class IndexGeneratorJob implements Jobby
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
{
final Configuration conf = JobHelper.injectSystemProperties(new Configuration());
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper;
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER;
ImmutableList.Builder<DataSegment> publishedSegmentsBuilder = ImmutableList.builder();
@ -176,18 +173,6 @@ public class IndexGeneratorJob implements Jobby
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);
JobHelper.setupClasspath(
@ -277,7 +262,7 @@ public class IndexGeneratorJob implements Jobby
final long truncatedTimestamp = granularitySpec.getQueryGranularity().truncate(inputRow.getTimestampFromEpoch());
final byte[] hashedDimensions = hashFunction.hashBytes(
HadoopDruidIndexerConfig.jsonMapper.writeValueAsBytes(
HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(
Rows.toGroupKey(
truncatedTimestamp,
inputRow
@ -489,7 +474,7 @@ public class IndexGeneratorJob implements Jobby
final ProgressIndicator progressIndicator
) throws IOException
{
return IndexMaker.persist(
return HadoopDruidIndexerConfig.INDEX_MAKER.persist(
index, interval, file, null, config.getIndexSpec(), progressIndicator
);
}
@ -501,7 +486,7 @@ public class IndexGeneratorJob implements Jobby
ProgressIndicator progressIndicator
) throws IOException
{
return IndexMaker.mergeQueryableIndex(
return HadoopDruidIndexerConfig.INDEX_MAKER.mergeQueryableIndex(
indexes, aggs, file, config.getIndexSpec(), progressIndicator
);
}
@ -614,7 +599,7 @@ public class IndexGeneratorJob implements Jobby
}
for (File file : toMerge) {
indexes.add(IndexIO.loadIndex(file));
indexes.add(HadoopDruidIndexerConfig.INDEX_IO.loadIndex(file));
}
mergedBase = mergeQueryableIndex(
indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator

View File

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

View File

@ -19,8 +19,6 @@ package io.druid.indexer;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.BaseProgressIndicator;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
import io.druid.segment.ProgressIndicator;
import io.druid.segment.QueryableIndex;
import io.druid.segment.incremental.IncrementalIndex;
@ -68,7 +66,7 @@ public class LegacyIndexGeneratorJob extends IndexGeneratorJob
IncrementalIndex index, Interval interval, File file, ProgressIndicator progressIndicator
) 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
@ -79,7 +77,7 @@ public class LegacyIndexGeneratorJob extends IndexGeneratorJob
ProgressIndicator progressIndicator
) 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 String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable();
handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.jsonMapper);
handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.JSON_MAPPER);
return true;
}

View File

@ -26,7 +26,6 @@ import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import io.druid.data.input.InputRow;
import io.druid.indexer.HadoopDruidIndexerConfig;
import io.druid.timeline.DataSegment;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputFormat;
@ -55,7 +54,7 @@ public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow>
Configuration conf = context.getConfiguration();
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,
new TypeReference<List<WindowedDataSegment>>()
{

View File

@ -70,13 +70,13 @@ public class DatasourceInputSplit extends InputSplit implements Writable
@Override
public void write(DataOutput out) throws IOException
{
out.writeUTF(HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segments));
out.writeUTF(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(segments));
}
@Override
public void readFields(DataInput in) throws IOException
{
segments = HadoopDruidIndexerConfig.jsonMapper.readValue(
segments = HadoopDruidIndexerConfig.JSON_MAPPER.readValue(
in.readUTF(),
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.indexer.HadoopDruidIndexerConfig;
import io.druid.indexer.JobHelper;
import io.druid.segment.IndexIO;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexStorageAdapter;
import io.druid.segment.realtime.firehose.IngestSegmentFirehose;
@ -67,7 +66,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
@Override
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();
@ -91,7 +90,7 @@ public class DatasourceRecordReader extends RecordReader<NullWritable, InputRow>
JobHelper.unzipNoGuava(path, context.getConfiguration(), dir, context);
logger.info("finished fetching segment files");
QueryableIndex index = IndexIO.loadIndex(dir);
QueryableIndex index = HadoopDruidIndexerConfig.INDEX_IO.loadIndex(dir);
indexes.add(index);
numRows += index.getNumRows();

View File

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

View File

@ -34,6 +34,8 @@ import io.druid.guice.GuiceInjectors;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
import io.druid.server.DruidNode;
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 ObjectMapper jsonMapper;
public static final IndexIO INDEX_IO;
public static final IndexMerger INDEX_MERGER;
private static final Injector injector = Initialization.makeInjectorWithModules(
GuiceInjectors.makeStartupInjector(),
ImmutableList.<Module>of(
@ -68,6 +73,8 @@ public class HadoopDruidConverterConfig
static {
jsonMapper = injector.getInstance(ObjectMapper.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>>()

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -339,7 +339,7 @@ public class IndexGeneratorJobTest
@Before
public void setUp() throws Exception
{
mapper = HadoopDruidIndexerConfig.jsonMapper;
mapper = HadoopDruidIndexerConfig.JSON_MAPPER;
mapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed"));
mapper.registerSubtypes(new NamedType(SingleDimensionShardSpec.class, "single"));
@ -412,7 +412,7 @@ public class IndexGeneratorJobTest
List<ShardSpec> specs = Lists.newArrayList();
if (partitionType.equals("hashed")) {
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")) {
int partitionNum = 0;

View File

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

View File

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

View File

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

View File

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

View File

@ -19,6 +19,7 @@ package io.druid.indexing.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
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.task.Task;
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.DataSegmentMover;
@ -70,6 +74,9 @@ public class TaskToolbox
private final SegmentLoader segmentLoader;
private final ObjectMapper objectMapper;
private final File taskWorkDir;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
public TaskToolbox(
TaskConfig config,
@ -87,7 +94,10 @@ public class TaskToolbox
MonitorScheduler monitorScheduler,
SegmentLoader segmentLoader,
ObjectMapper objectMapper,
final File taskWorkDir
File taskWorkDir,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
)
{
this.config = config;
@ -106,6 +116,9 @@ public class TaskToolbox
this.segmentLoader = segmentLoader;
this.objectMapper = objectMapper;
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()
@ -208,4 +221,19 @@ public class TaskToolbox
{
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;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.metamx.emitter.service.ServiceEmitter;
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.task.Task;
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.DataSegmentMover;
@ -55,6 +59,9 @@ public class TaskToolboxFactory
private final MonitorScheduler monitorScheduler;
private final SegmentLoaderFactory segmentLoaderFactory;
private final ObjectMapper objectMapper;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@Inject
public TaskToolboxFactory(
@ -71,7 +78,10 @@ public class TaskToolboxFactory
@Processing ExecutorService queryExecutorService,
MonitorScheduler monitorScheduler,
SegmentLoaderFactory segmentLoaderFactory,
ObjectMapper objectMapper
ObjectMapper objectMapper,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
)
{
this.config = config;
@ -88,6 +98,9 @@ public class TaskToolboxFactory
this.monitorScheduler = monitorScheduler;
this.segmentLoaderFactory = segmentLoaderFactory;
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)
@ -110,7 +123,10 @@ public class TaskToolboxFactory
monitorScheduler,
segmentLoaderFactory.manufacturate(taskWorkDir),
objectMapper,
taskWorkDir
taskWorkDir,
indexMerger,
indexMaker,
indexIO
);
}
}

View File

@ -65,6 +65,8 @@ public class YeOldePlumberSchool implements PlumberSchool
private final String version;
private final DataSegmentPusher dataSegmentPusher;
private final File tmpSegmentDir;
private final IndexMerger indexMerger;
private final IndexIO indexIO;
private static final Logger log = new Logger(YeOldePlumberSchool.class);
@ -73,13 +75,17 @@ public class YeOldePlumberSchool implements PlumberSchool
@JsonProperty("interval") Interval interval,
@JsonProperty("version") String version,
@JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher,
@JacksonInject("tmpSegmentDir") File tmpSegmentDir
@JacksonInject("tmpSegmentDir") File tmpSegmentDir,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexIO indexIO
)
{
this.interval = interval;
this.version = version;
this.dataSegmentPusher = dataSegmentPusher;
this.tmpSegmentDir = tmpSegmentDir;
this.indexMerger = Preconditions.checkNotNull(indexMerger, "Null IndexMerger");
this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO");
}
@Override
@ -162,15 +168,15 @@ public class YeOldePlumberSchool implements PlumberSchool
} else {
List<QueryableIndex> indexes = Lists.newArrayList();
for (final File oneSpill : spilled) {
indexes.add(IndexIO.loadIndex(oneSpill));
indexes.add(indexIO.loadIndex(oneSpill));
}
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
final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload);
final QueryableIndex mappedSegment = indexIO.loadIndex(fileToUpload);
final DataSegment segmentToUpload = theSink.getSegment()
.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);
try {
IndexMerger.persist(
indexMerger.persist(
indexToPersist.getIndex(),
dirToPersist,
null,

View File

@ -25,8 +25,7 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMerger;
import io.druid.indexing.common.TaskToolbox;
import io.druid.segment.IndexSpec;
import io.druid.segment.IndexableAdapter;
import io.druid.segment.QueryableIndexIndexableAdapter;
@ -64,7 +63,7 @@ public class AppendTask extends MergeTaskBase
}
@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
{
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
@ -113,7 +112,7 @@ public class AppendTask extends MergeTaskBase
adapters.add(
new RowboatFilteringIndexAdapter(
new QueryableIndexIndexableAdapter(
IndexIO.loadIndex(holder.getFile())
toolbox.getIndexIO().loadIndex(holder.getFile())
),
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

View File

@ -39,7 +39,6 @@ import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
@ -99,7 +98,12 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
*
* @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();
@ -377,7 +381,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
final File location = localSegments.get(segment);
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);
// 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];
String version = args[1];
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.jsonMapper
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER
.readValue(
schema,
HadoopIngestionSpec.class
@ -257,7 +257,7 @@ public class HadoopIndexTask extends HadoopTask
log.info("Starting a hadoop index generator job...");
if (job.run()) {
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(job.getPublishedSegments());
return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(job.getPublishedSegments());
}
return null;
@ -272,7 +272,7 @@ public class HadoopIndexTask extends HadoopTask
final String workingPath = args[1];
final String segmentOutputPath = args[2];
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.jsonMapper
final HadoopIngestionSpec theSchema = HadoopDruidIndexerConfig.JSON_MAPPER
.readValue(
schema,
HadoopIngestionSpec.class
@ -287,7 +287,7 @@ public class HadoopIndexTask extends HadoopTask
log.info("Starting a hadoop determine configuration job...");
if (job.run()) {
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(config.getSchema());
return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config.getSchema());
}
return null;

View File

@ -61,7 +61,6 @@ import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.HashBasedNumberedShardSpec;
import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.ShardSpec;
import java.util.Map;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -69,6 +68,7 @@ import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.concurrent.CopyOnWriteArrayList;
@ -359,7 +359,9 @@ public class IndexTask extends AbstractFixedIntervalTask
interval,
version,
wrappedDataSegmentPusher,
tmpDir
tmpDir,
toolbox.getIndexMerger(),
toolbox.getIndexIO()
).findPlumber(
schema,
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.collect.ImmutableList;
import com.google.common.collect.Lists;
import io.druid.indexing.common.TaskToolbox;
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.QueryableIndex;
import io.druid.timeline.DataSegment;
@ -53,7 +52,6 @@ public class MergeTask extends MergeTaskBase
@JsonProperty("aggregations") List<AggregatorFactory> aggregators,
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("context") Map<String, Object> context
)
{
super(id, dataSource, segments, context);
@ -62,10 +60,10 @@ public class MergeTask extends MergeTaskBase
}
@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
{
return IndexMerger.mergeQueryableIndex(
return toolbox.getIndexMerger().mergeQueryableIndex(
Lists.transform(
ImmutableList.copyOf(segments.values()),
new Function<File, QueryableIndex>()
@ -74,7 +72,7 @@ public class MergeTask extends MergeTaskBase
public QueryableIndex apply(@Nullable File input)
{
try {
return IndexIO.loadIndex(input);
return toolbox.getIndexIO().loadIndex(input);
}
catch (Exception 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);
// 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/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;
@JsonProperty

View File

@ -275,7 +275,10 @@ public class RealtimeIndexTask extends AbstractTask
lockingSegmentAnnouncer,
segmentPublisher,
toolbox.getNewSegmentServerView(),
toolbox.getQueryExecutorService()
toolbox.getQueryExecutorService(),
toolbox.getIndexMerger(),
toolbox.getIndexMaker(),
toolbox.getIndexIO()
);
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> metrics;
private final Injector injector;
private final IndexIO indexIO;
@JsonCreator
public IngestSegmentFirehoseFactory(
@ -73,7 +74,8 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
@JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("dimensions") List<String> dimensions,
@JsonProperty("metrics") List<String> metrics,
@JacksonInject Injector injector
@JacksonInject Injector injector,
@JacksonInject IndexIO indexIO
)
{
Preconditions.checkNotNull(dataSource, "dataSource");
@ -84,6 +86,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
this.dimensions = dimensions;
this.metrics = metrics;
this.injector = injector;
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
}
@JsonProperty
@ -247,7 +250,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
try {
return new WindowedStorageAdapter(
new QueryableIndexStorageAdapter(
IndexIO.loadIndex(
indexIO.loadIndex(
Preconditions.checkNotNull(
segmentFileMap.get(segment),
"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.task.Task;
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.DataSegmentMover;
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.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.timeline.DataSegment;
import org.easymock.EasyMock;
@ -70,6 +73,9 @@ public class TaskToolboxTest
private ObjectMapper ObjectMapper = new ObjectMapper();
private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.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
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@ -94,7 +100,10 @@ public class TaskToolboxTest
mockQueryExecutorService,
mockMonitorScheduler,
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 io.druid.indexing.common.task.RealtimeIndexTask;
import io.druid.indexing.common.task.TaskResource;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;

View File

@ -17,8 +17,6 @@
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.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
@ -26,6 +24,10 @@ import com.google.common.base.Stopwatch;
import com.metamx.common.ISE;
import io.druid.guice.ServerModule;
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.concurrent.TimeUnit;
@ -34,28 +36,60 @@ import java.util.concurrent.TimeUnit;
*/
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 {
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {
MAPPER.registerModule(module);
}
MAPPER.setInjectableValues(
new InjectableValues()
public TestUtils()
{
jsonMapper = new DefaultObjectMapper();
indexIO = new IndexIO(
jsonMapper,
new ColumnConfig()
{
@Override
public Object findInjectableValue(
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
)
public int columnCacheSizeBytes()
{
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
return TestUtils.MAPPER;
}
throw new ISE("No Injectable value found");
return 0;
}
}
);
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)

View File

@ -17,13 +17,12 @@
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.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.indexing.common.TestUtils;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import java.io.IOException;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
@ -33,7 +32,13 @@ import org.junit.Test;
*/
public class ConvertSegmentTaskTest
{
private DefaultObjectMapper jsonMapper = new DefaultObjectMapper();
private final ObjectMapper jsonMapper;
public ConvertSegmentTaskTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test
public void testSerializationSimple() throws Exception
@ -41,7 +46,6 @@ public class ConvertSegmentTaskTest
final String dataSource = "billy";
final Interval interval = new Interval(new DateTime().minus(1000), new DateTime());
ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null);
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.google.common.collect.ImmutableList;
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.data.ConciseBitmapSerdeFactory;
import io.druid.timeline.DataSegment;
@ -39,7 +39,7 @@ import java.util.Map;
public class HadoopConverterTaskSerDeTest
{
private static ObjectMapper objectMapper = new DefaultObjectMapper();
private static final String TASK_ID = "task id";
private static final String DATA_SOURCE = "datasource";
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 CLASSPATH_PREFIX = "something:where:I:need:stuff";
private final ObjectMapper jsonMapper;
public HadoopConverterTaskSerDeTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test
public void testSimpleConverterTaskSerDe() throws IOException
{
@ -85,9 +93,9 @@ public class HadoopConverterTaskSerDeTest
CLASSPATH_PREFIX,
null
);
final String strOrig = objectMapper.writeValueAsString(orig);
HadoopConverterTask other = objectMapper.readValue(strOrig, HadoopConverterTask.class);
Assert.assertEquals(strOrig, objectMapper.writeValueAsString(other));
final String strOrig = jsonMapper.writeValueAsString(orig);
HadoopConverterTask other = jsonMapper.readValue(strOrig, HadoopConverterTask.class);
Assert.assertEquals(strOrig, jsonMapper.writeValueAsString(other));
Assert.assertFalse(orig == other);
Assert.assertEquals(orig, other);
assertExpectedTask(other);
@ -117,13 +125,13 @@ public class HadoopConverterTaskSerDeTest
parent,
null
);
final String origString = objectMapper.writeValueAsString(subTask);
final HadoopConverterTask.ConverterSubTask otherSub = objectMapper.readValue(
final String origString = jsonMapper.writeValueAsString(subTask);
final HadoopConverterTask.ConverterSubTask otherSub = jsonMapper.readValue(
origString,
HadoopConverterTask.ConverterSubTask.class
);
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.assertFalse(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.indexing.common.TaskLock;
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.TaskAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
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.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeTuningConfig;
@ -63,8 +66,21 @@ public class IndexTaskTest
{
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private final IndexSpec indexSpec = new IndexSpec();
private final ObjectMapper jsonMapper = new DefaultObjectMapper();
private final IndexSpec indexSpec;
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
public void testDeterminePartitions() throws Exception
@ -128,7 +144,7 @@ public class IndexTaskTest
indexSpec
)
),
new DefaultObjectMapper(),
jsonMapper,
null
);
@ -193,7 +209,7 @@ public class IndexTaskTest
),
null
),
new DefaultObjectMapper(),
jsonMapper,
null
);
@ -243,7 +259,8 @@ public class IndexTaskTest
segments.add(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
),
new DefaultObjectMapper(),
jsonMapper,
null
);

View File

@ -20,6 +20,7 @@ package io.druid.indexing.common.task;
import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableList;
import com.google.common.hash.Hashing;
import io.druid.indexing.common.TaskToolbox;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert;
@ -44,7 +45,7 @@ public class MergeTaskBaseTest
final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments, null)
{
@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;
}

View File

@ -19,7 +19,6 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
@ -29,7 +28,7 @@ import io.druid.granularity.QueryGranularity;
import io.druid.guice.FirehoseModule;
import io.druid.indexer.HadoopIOConfig;
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.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
@ -56,14 +55,19 @@ import java.io.IOException;
public class TaskSerdeTest
{
private static final ObjectMapper jsonMapper;
static {
jsonMapper = new DefaultObjectMapper();
jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper));
}
private final ObjectMapper jsonMapper;
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
public void testIndexTaskSerde() throws Exception
{
@ -89,15 +93,10 @@ public class TaskSerdeTest
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);
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(new Interval("2010-01-01/P2D"), task.getInterval());
@ -138,12 +137,11 @@ public class TaskSerdeTest
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);
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(new Interval("2010-01-01/P2D"), task.getInterval());
@ -554,9 +552,7 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
InjectableValues inject = new InjectableValues.Std()
.addValue(ObjectMapper.class, jsonMapper);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.reader(Task.class).with(inject).readValue(json);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());

View File

@ -48,18 +48,20 @@ import io.druid.granularity.QueryGranularity;
import io.druid.guice.GuiceInjectors;
import io.druid.indexing.common.SegmentLoaderFactory;
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.TaskActionToolbox;
import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
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.IndexSpec;
import io.druid.segment.incremental.IncrementalIndexSchema;
@ -102,6 +104,18 @@ import java.util.Set;
@RunWith(Parameterized.class)
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}")
public static Collection<Object[]> constructorFeeder() throws IOException
@ -136,7 +150,7 @@ public class IngestSegmentFirehoseFactoryTest
if (!persistDir.mkdirs() && !persistDir.exists()) {
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 IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null)
@ -179,7 +193,7 @@ public class IngestSegmentFirehoseFactoryTest
ts,
new TaskActionToolbox(tl, mdc, newMockEmitter())
);
final ObjectMapper objectMapper = newObjectMapper();
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null),
tac,
@ -244,10 +258,13 @@ public class IngestSegmentFirehoseFactoryTest
{
return Lists.newArrayList();
}
}, objectMapper
}, MAPPER
)
),
objectMapper
MAPPER,
INDEX_MERGER,
INDEX_MAKER,
INDEX_IO
);
Collection<Object[]> values = new LinkedList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList(
@ -285,7 +302,8 @@ public class IngestSegmentFirehoseFactoryTest
binder.bind(TaskToolboxFactory.class).toInstance(taskToolboxFactory);
}
}
)
),
INDEX_IO
),
String.format(
"DimNames[%s]MetricNames[%s]ParserDimNames[%s]",
@ -302,9 +320,8 @@ public class IngestSegmentFirehoseFactoryTest
return values;
}
public static ObjectMapper newObjectMapper()
public static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper)
{
final ObjectMapper objectMapper = new DefaultObjectMapper();
objectMapper.registerModule(
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.indexing.common.SegmentLoaderFactory;
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.TaskAction;
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.LongSumAggregatorFactory;
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.IndexSpec;
import io.druid.segment.incremental.IncrementalIndexSchema;
@ -104,6 +107,19 @@ public class IngestSegmentFirehoseFactoryTimelineTest
private final int expectedCount;
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(
String name,
IngestSegmentFirehoseFactory factory,
@ -212,7 +228,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
}
try {
IndexMerger.persist(index, persistDir, null, new IndexSpec());
INDEX_MERGER.persist(index, persistDir, null, new IndexSpec());
}
catch (IOException e) {
throw Throwables.propagate(e);
@ -262,7 +278,6 @@ public class IngestSegmentFirehoseFactoryTimelineTest
final List<Object[]> constructors = Lists.newArrayList();
for (final TestCase testCase : testCases) {
final ObjectMapper objectMapper = IngestSegmentFirehoseFactoryTest.newObjectMapper();
final TaskActionClient taskActionClient = new TaskActionClient()
{
@Override
@ -311,10 +326,13 @@ public class IngestSegmentFirehoseFactoryTimelineTest
{
return Lists.newArrayList();
}
}, objectMapper
}, MAPPER
)
),
objectMapper
MAPPER,
INDEX_MERGER,
INDEX_MAKER,
INDEX_IO
);
final Injector injector = Guice.createInjector(
new Module()
@ -332,7 +350,8 @@ public class IngestSegmentFirehoseFactoryTimelineTest
new NoopDimFilter(),
Arrays.asList(DIMENSIONS),
Arrays.asList(METRICS),
injector
injector,
INDEX_IO
);
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.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.server.initialization.IndexerZkConfig;
import io.druid.server.initialization.ZkPathsConfig;
import org.apache.curator.framework.CuratorFramework;
@ -66,7 +65,6 @@ import java.util.concurrent.atomic.AtomicReference;
public class RemoteTaskRunnerTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid";
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 int TIMEOUT_SECONDS = 5;
private ObjectMapper jsonMapper;
private TestingCluster testingCluster;
private CuratorFramework cf;
private RemoteTaskRunner remoteTaskRunner;
@ -85,6 +85,9 @@ public class RemoteTaskRunnerTest
@Before
public void setUp() throws Exception
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
testingCluster = new TestingCluster(1);
testingCluster.start();

View File

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

View File

@ -19,6 +19,7 @@ package io.druid.indexing.worker;
import com.fasterxml.jackson.databind.ObjectMapper;
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.Task;
import io.druid.indexing.common.task.TaskResource;
@ -39,6 +40,14 @@ import java.io.File;
public class TaskAnnouncementTest
{
private final ObjectMapper jsonMapper;
public TaskAnnouncementTest()
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
}
@Test
public void testBackwardsCompatibleSerde() throws Exception
{
@ -68,7 +77,6 @@ public class TaskAnnouncementTest
final TaskStatus status = TaskStatus.running(task.getId());
final TaskAnnouncement announcement = TaskAnnouncement.create(task, status);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String statusJson = jsonMapper.writeValueAsString(status);
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.ThreadPoolTaskRunner;
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.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.StorageLocationConfig;
@ -56,8 +58,6 @@ import java.util.List;
*/
public class WorkerTaskMonitorTest
{
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private static final Joiner joiner = Joiner.on("/");
private static final String basePath = "/test/druid";
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
@ -71,6 +71,19 @@ public class WorkerTaskMonitorTest
private TestMergeTask task;
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
public void setUp() throws Exception
@ -143,7 +156,11 @@ public class WorkerTaskMonitorTest
}
, jsonMapper
)
), jsonMapper
),
jsonMapper,
indexMerger,
indexMaker,
indexIO
),
null
),

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
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.Files;
import com.google.common.primitives.Ints;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.google.inject.Inject;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ConciseBitmapFactory;
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.emitter.EmittingLogger;
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.ColumnBuilder;
import io.druid.segment.column.ColumnConfig;
@ -113,64 +106,116 @@ public class IndexIO
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
private static 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 final Map<Integer, IndexLoader> indexLoaders;
private static final EmittingLogger log = new EmittingLogger(IndexIO.class);
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 IndexLoader loader = indexLoaders.get(version);
if (loader != null) {
return loader.load(inDir);
return loader.load(inDir, mapper);
} else {
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);
}
public static boolean convertSegment(
public boolean convertSegment(
File toConvert,
File converted,
IndexSpec indexSpec,
boolean forceIfCurrent,
boolean validate
)
throws IOException
throws IOException
{
final int version = SegmentUtils.getVersionFromDir(toConvert);
switch (version) {
@ -227,20 +272,20 @@ public class IndexIO
case 6:
case 7:
log.info("Old version, re-persisting.");
IndexMerger.append(
new IndexMerger(mapper, this).append(
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
converted,
indexSpec
);
return true;
case 8:
DefaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec);
defaultIndexIOHandler.convertV8toV9(toConvert, converted, indexSpec);
return true;
default:
if (forceIfCurrent) {
IndexMerger.convert(toConvert, converted, indexSpec);
new IndexMerger(mapper, this).convert(toConvert, converted, indexSpec);
if (validate) {
DefaultIndexIOHandler.validateTwoSegments(toConvert, converted);
validateTwoSegments(toConvert, converted);
}
return true;
} 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;
}
@ -370,6 +420,12 @@ public class IndexIO
public static class DefaultIndexIOHandler implements IndexIOHandler
{
private static final Logger log = new Logger(DefaultIndexIOHandler.class);
private final ObjectMapper mapper;
public DefaultIndexIOHandler(ObjectMapper mapper)
{
this.mapper = mapper;
}
@Override
public MMappedIndex mapDir(File inDir) throws IOException
@ -473,78 +529,8 @@ public class IndexIO
return retVal;
}
public static 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 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
public void convertV8toV9(File v8Dir, File v9Dir, IndexSpec indexSpec)
throws IOException
{
log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir);
@ -888,15 +874,22 @@ public class IndexIO
static interface IndexLoader
{
public QueryableIndex load(File inDir) throws IOException;
public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException;
}
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
public QueryableIndex load(File inDir) throws IOException
public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException
{
MMappedIndex index = legacyHandler.mapDir(inDir);
@ -990,8 +983,15 @@ public class IndexIO
static class V9IndexLoader implements IndexLoader
{
private final ColumnConfig columnConfig;
V9IndexLoader(ColumnConfig columnConfig)
{
this.columnConfig = columnConfig;
}
@Override
public QueryableIndex load(File inDir) throws IOException
public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException
{
log.debug("Mapping v9 index[%s]", inDir);
long startTime = System.currentTimeMillis();

View File

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

View File

@ -22,6 +22,7 @@ package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
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.OutputSupplier;
import com.google.common.primitives.Ints;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Module;
import com.google.inject.Inject;
import com.metamx.collections.bitmap.BitmapFactory;
import com.metamx.collections.bitmap.ImmutableBitmap;
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.utils.JodaUtils;
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.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl;
@ -109,26 +106,21 @@ public class IndexMerger
private static final int INVALID_ROW = -1;
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 static File persist(
public File persist(
final IncrementalIndex index,
File outDir,
Map<String, Object> segmentMetadata,
@ -150,7 +142,7 @@ public class IndexMerger
*
* @throws java.io.IOException if an IO error occurs persisting the index
*/
public static File persist(
public File persist(
final IncrementalIndex index,
final Interval dataInterval,
File outDir,
@ -161,7 +153,7 @@ public class IndexMerger
return persist(index, dataInterval, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator());
}
public static File persist(
public File persist(
final IncrementalIndex index,
final Interval dataInterval,
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
) throws IOException
{
return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
}
public static File mergeQueryableIndex(
public File mergeQueryableIndex(
List<QueryableIndex> indexes,
final AggregatorFactory[] metricAggs,
File outDir,
@ -244,7 +236,7 @@ public class IndexMerger
);
}
public static File merge(
public File merge(
List<IndexableAdapter> indexes,
final AggregatorFactory[] metricAggs,
File outDir,
@ -255,7 +247,7 @@ public class IndexMerger
return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator());
}
public static File merge(
public File merge(
List<IndexableAdapter> indexes,
final AggregatorFactory[] metricAggs,
File outDir,
@ -361,16 +353,16 @@ public class IndexMerger
}
// 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());
}
public static File convert(
public File convert(
final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress
) throws IOException
{
try (QueryableIndex index = IndexIO.loadIndex(inDir)) {
try (QueryableIndex index = indexIO.loadIndex(inDir)) {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
return makeIndexFiles(
ImmutableList.of(adapter),
@ -393,14 +385,14 @@ public class IndexMerger
}
}
public static File append(
public File append(
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec
) throws IOException
{
return append(indexes, outDir, indexSpec, new BaseProgressIndicator());
}
public static File append(
public File append(
List<IndexableAdapter> indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress
) throws IOException
{
@ -473,7 +465,7 @@ public class IndexMerger
return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec);
}
private static File makeIndexFiles(
private File makeIndexFiles(
final List<IndexableAdapter> indexes,
final File outDir,
final ProgressIndicator progress,
@ -954,13 +946,13 @@ public class IndexMerger
indexSpec.getBitmapSerdeFactory()
);
IndexIO.DefaultIndexIOHandler.convertV8toV9(v8OutDir, outDir, indexSpec);
indexIO.getDefaultIndexIOHandler().convertV8toV9(v8OutDir, outDir, indexSpec);
FileUtils.deleteDirectory(v8OutDir);
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());
@ -973,7 +965,7 @@ public class IndexMerger
return Lists.newArrayList(retVal);
}
public static void createIndexDrdFile(
public void createIndexDrdFile(
byte versionId,
File inDir,
GenericIndexed<String> availableDimensions,
@ -1292,7 +1284,7 @@ public class IndexMerger
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);
FileChannel metadataFilechannel = metadataFileOutputStream.getChannel()

View File

@ -27,6 +27,7 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -98,7 +99,7 @@ public class IndexSpec
Preconditions.checkArgument(metricCompression == null || COMPRESSION_NAMES.contains(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.dimensionCompression = dimensionCompression;
}

View File

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

View File

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

View File

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

View File

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

View File

@ -67,6 +67,11 @@ public class IndexMakerTest
"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(
new ConciseBitmapSerdeFactory(),
CompressedObjectStrategy.CompressionStrategy.LZ4,
@ -170,7 +175,7 @@ public class IndexMakerTest
}
tmpDir = Files.createTempDir();
persistTmpDir = new File(tmpDir, "persistDir");
IndexMerger.persist(toPersist, persistTmpDir, null, INDEX_SPEC);
INDEX_MERGER.persist(toPersist, persistTmpDir, null, INDEX_SPEC);
}
@After
@ -187,7 +192,7 @@ public class IndexMakerTest
try {
outDir = Files.createTempDir();
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());
}
@ -208,7 +213,7 @@ public class IndexMakerTest
{
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater(
IndexIO.loadIndex(
INDEX_IO.loadIndex(
persistTmpDir
)
)
@ -219,23 +224,23 @@ public class IndexMakerTest
private File reprocessAndValidate(File inDir, File tmpDir) throws IOException
{
final File outDir = IndexMaker.convert(
final File outDir = INDEX_MAKER.convert(
inDir,
tmpDir,
INDEX_SPEC
);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(persistTmpDir, outDir);
INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir;
}
private File appendAndValidate(File inDir, File tmpDir) throws IOException
{
final File outDir = IndexMerger.append(
ImmutableList.<IndexableAdapter>of(new QueryableIndexIndexableAdapter(closer.closeLater(IndexIO.loadIndex(inDir)))),
final File outDir = INDEX_MERGER.append(
ImmutableList.<IndexableAdapter>of(new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(inDir)))),
tmpDir,
INDEX_SPEC
);
IndexIO.DefaultIndexIOHandler.validateTwoSegments(persistTmpDir, outDir);
INDEX_IO.validateTwoSegments(persistTmpDir, outDir);
return outDir;
}
@ -244,7 +249,7 @@ public class IndexMakerTest
{
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater(
IndexIO.loadIndex(
INDEX_IO.loadIndex(
persistTmpDir
)
)
@ -254,12 +259,12 @@ public class IndexMakerTest
reprocessAndValidate(persistTmpDir, tmpDir1);
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());
reprocessAndValidate(tmpDir1, tmpDir2);
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());
reprocessAndValidate(tmpDir2, tmpDir3);
}
@ -269,7 +274,7 @@ public class IndexMakerTest
{
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater(
IndexIO.loadIndex(
INDEX_IO.loadIndex(
persistTmpDir
)
)
@ -283,7 +288,7 @@ public class IndexMakerTest
{
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(
closer.closeLater(
IndexIO.loadIndex(
INDEX_IO.loadIndex(
persistTmpDir
)
)
@ -293,12 +298,12 @@ public class IndexMakerTest
appendAndValidate(persistTmpDir, tmpDir1);
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());
appendAndValidate(tmpDir1, tmpDir2);
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());
appendAndValidate(tmpDir2, tmpDir3);
}

View File

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

View File

@ -29,13 +29,6 @@ import org.junit.Test;
public class IndexSpecTest
{
@Test
public void testConfiguredBitmap() throws Exception
{
// this is just to make sure testSerde correctly tests the bitmap type override
Assert.assertEquals(new ConciseBitmapSerdeFactory(), IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY);
}
@Test
public void testSerde() throws Exception
{
@ -67,7 +60,6 @@ public class IndexSpecTest
public void testDefaults() throws Exception
{
final IndexSpec spec = new IndexSpec();
Assert.assertEquals(IndexIO.CONFIGURED_BITMAP_SERDE_FACTORY, spec.getBitmapSerdeFactory());
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompressionStrategy());
Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompressionStrategy());
}

View File

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

View File

@ -17,11 +17,14 @@
package io.druid.segment;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import io.druid.data.input.Row;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Result;
import io.druid.segment.column.ColumnConfig;
import org.junit.Assert;
import java.util.Iterator;
@ -30,6 +33,42 @@ import java.util.Iterator;
*/
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)
{
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 IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger();
private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO();
static {
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
@ -142,12 +145,12 @@ public class TestIndex
mergedFile.mkdirs();
mergedFile.deleteOnExit();
IndexMerger.persist(top, DATA_INTERVAL, topFile, null, indexSpec);
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile, null, indexSpec);
INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, null, indexSpec);
INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, null, indexSpec);
mergedRealtime = IndexIO.loadIndex(
IndexMerger.mergeQueryableIndex(
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)),
mergedRealtime = INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)),
METRIC_AGGS,
mergedFile,
indexSpec
@ -252,8 +255,8 @@ public class TestIndex
someTmpFile.mkdirs();
someTmpFile.deleteOnExit();
IndexMerger.persist(index, someTmpFile, null, indexSpec);
return IndexIO.loadIndex(someTmpFile);
INDEX_MERGER.persist(index, someTmpFile, null, indexSpec);
return INDEX_IO.loadIndex(someTmpFile);
}
catch (IOException e) {
throw Throwables.propagate(e);

View File

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

View File

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

View File

@ -17,6 +17,8 @@
package io.druid.segment.loading;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.segment.IndexIO;
import io.druid.segment.QueryableIndex;
@ -30,11 +32,19 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory
{
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
public QueryableIndex factorize(File parentDir) throws SegmentLoadingException
{
try {
return IndexIO.loadIndex(parentDir);
return indexIO.loadIndex(parentDir);
}
catch (IOException e) {
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.concurrent.Execs;
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.RealtimeTuningConfig;
import io.druid.segment.realtime.FireDepartmentMetrics;
@ -59,7 +62,10 @@ public class FlushingPlumber extends RealtimePlumber
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
DataSegmentAnnouncer segmentAnnouncer,
ExecutorService queryExecutorService
ExecutorService queryExecutorService,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
)
{
super(
@ -72,7 +78,10 @@ public class FlushingPlumber extends RealtimePlumber
queryExecutorService,
null,
null,
null
null,
indexMerger,
indexMaker,
indexIO
);
this.flushDuration = flushDuration;

View File

@ -24,6 +24,9 @@ import com.google.common.base.Preconditions;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.guice.annotations.Processing;
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.RealtimeTuningConfig;
import io.druid.segment.realtime.FireDepartmentMetrics;
@ -46,6 +49,9 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
private final QueryRunnerFactoryConglomerate conglomerate;
private final DataSegmentAnnouncer segmentAnnouncer;
private final ExecutorService queryExecutorService;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@JsonCreator
public FlushingPlumberSchool(
@ -53,7 +59,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
@JacksonInject ServiceEmitter emitter,
@JacksonInject QueryRunnerFactoryConglomerate conglomerate,
@JacksonInject DataSegmentAnnouncer segmentAnnouncer,
@JacksonInject @Processing ExecutorService queryExecutorService
@JacksonInject @Processing ExecutorService queryExecutorService,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexMaker indexMaker,
@JacksonInject IndexIO indexIO
)
{
super(
@ -63,7 +72,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
segmentAnnouncer,
null,
null,
queryExecutorService
queryExecutorService,
indexMerger,
indexMaker,
indexIO
);
this.flushDuration = flushDuration == null ? defaultFlushDuration : flushDuration;
@ -71,6 +83,9 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
this.conglomerate = conglomerate;
this.segmentAnnouncer = segmentAnnouncer;
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
@ -90,7 +105,10 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool
emitter,
conglomerate,
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.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Stopwatch;
import com.google.common.base.Supplier;
@ -129,6 +130,9 @@ public class RealtimePlumber implements Plumber
private volatile ExecutorService persistExecutor = null;
private volatile ExecutorService mergeExecutor = 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_TIMESTAMP_KEY = "%commitMetadataTimestamp%";
@ -143,7 +147,10 @@ public class RealtimePlumber implements Plumber
ExecutorService queryExecutorService,
DataSegmentPusher dataSegmentPusher,
SegmentPublisher segmentPublisher,
FilteredServerView serverView
FilteredServerView serverView,
IndexMerger indexMerger,
IndexMaker indexMaker,
IndexIO indexIO
)
{
this.schema = schema;
@ -157,6 +164,9 @@ public class RealtimePlumber implements Plumber
this.dataSegmentPusher = dataSegmentPusher;
this.segmentPublisher = segmentPublisher;
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());
}
@ -500,14 +510,14 @@ public class RealtimePlumber implements Plumber
final File mergedFile;
if (config.isPersistInHeap()) {
mergedFile = IndexMaker.mergeQueryableIndex(
mergedFile = indexMaker.mergeQueryableIndex(
indexes,
schema.getAggregators(),
mergedTarget,
config.getIndexSpec()
);
} else {
mergedFile = IndexMerger.mergeQueryableIndex(
mergedFile = indexMerger.mergeQueryableIndex(
indexes,
schema.getAggregators(),
mergedTarget,
@ -518,7 +528,7 @@ public class RealtimePlumber implements Plumber
metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime);
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());
DataSegment segment = dataSegmentPusher.push(
@ -705,7 +715,7 @@ public class RealtimePlumber implements Plumber
}
QueryableIndex queryableIndex = null;
try {
queryableIndex = IndexIO.loadIndex(segmentDir);
queryableIndex = indexIO.loadIndex(segmentDir);
}
catch (IOException e) {
log.error(e, "Problem loading segmentDir from disk.");
@ -974,14 +984,14 @@ public class RealtimePlumber implements Plumber
final IndexSpec indexSpec = config.getIndexSpec();
if (config.isPersistInHeap()) {
persistedFile = IndexMaker.persist(
persistedFile = indexMaker.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())),
metaData,
indexSpec
);
} else {
persistedFile = IndexMerger.persist(
persistedFile = indexMerger.persist(
indexToPersist.getIndex(),
new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())),
metaData,
@ -992,7 +1002,7 @@ public class RealtimePlumber implements Plumber
indexToPersist.swapSegment(
new QueryableIndexSegment(
indexToPersist.getSegment().getIdentifier(),
IndexIO.loadIndex(persistedFile)
indexIO.loadIndex(persistedFile)
)
);
return numRows;

View File

@ -24,6 +24,9 @@ import com.metamx.emitter.service.ServiceEmitter;
import io.druid.client.FilteredServerView;
import io.druid.guice.annotations.Processing;
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.RealtimeTuningConfig;
import io.druid.segment.loading.DataSegmentPusher;
@ -44,6 +47,9 @@ public class RealtimePlumberSchool implements PlumberSchool
private final SegmentPublisher segmentPublisher;
private final FilteredServerView serverView;
private final ExecutorService queryExecutorService;
private final IndexMerger indexMerger;
private final IndexMaker indexMaker;
private final IndexIO indexIO;
@JsonCreator
public RealtimePlumberSchool(
@ -53,7 +59,10 @@ public class RealtimePlumberSchool implements PlumberSchool
@JacksonInject DataSegmentAnnouncer segmentAnnouncer,
@JacksonInject SegmentPublisher segmentPublisher,
@JacksonInject FilteredServerView serverView,
@JacksonInject @Processing ExecutorService executorService
@JacksonInject @Processing ExecutorService executorService,
@JacksonInject IndexMerger indexMerger,
@JacksonInject IndexMaker indexMaker,
@JacksonInject IndexIO indexIO
)
{
this.emitter = emitter;
@ -63,6 +72,9 @@ public class RealtimePlumberSchool implements PlumberSchool
this.segmentPublisher = segmentPublisher;
this.serverView = serverView;
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
@ -84,7 +96,10 @@ public class RealtimePlumberSchool implements PlumberSchool
queryExecutorService,
dataSegmentPusher,
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.Lists;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.TestHelper;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.Interval;
@ -72,7 +73,7 @@ public class SegmentLoaderLocalCacheManagerTest
locations.add(locationConfig);
manager = new SegmentLoaderLocalCacheManager(
new MMappedQueryableIndexFactory(),
new MMappedQueryableIndexFactory(TestHelper.getTestIndexIO()),
new SegmentLoaderConfig().withLocations(locations),
jsonMapper
);

View File

@ -28,6 +28,7 @@ import io.druid.granularity.QueryGranularity;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.segment.TestHelper;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.RealtimeTuningConfig;
@ -78,7 +79,16 @@ public class FireDepartmentTest
new RealtimeIOConfig(
null,
new RealtimePlumberSchool(
null, null, null, null, null, null, null
null,
null,
null,
null,
null,
null,
null,
TestHelper.getTestIndexMerger(),
TestHelper.getTestIndexMaker(),
TestHelper.getTestIndexIO()
),
null
),

View File

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

View File

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

View File

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