mirror of https://github.com/apache/druid.git
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:
commit
cec09a9967
|
@ -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|
|
||||
|
||||
|
|
|
@ -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>>()
|
||||
{
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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++
|
||||
)
|
||||
);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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>>()
|
||||
{
|
||||
|
|
|
@ -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>>()
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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>>()
|
||||
|
|
|
@ -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
|
||||
)
|
||||
)
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()), "");
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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)
|
||||
)
|
||||
|
|
Loading…
Reference in New Issue