mirror of https://github.com/apache/druid.git
fixed createTable; fixed miscellaneous stuff; added DerbyMetadataRuleManagerProvider
This commit is contained in:
parent
76304d6b7b
commit
7d5c5f2083
|
@ -27,8 +27,8 @@ import com.metamx.common.concurrent.ScheduledExecutors;
|
|||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.db.MetadataStorageConnector;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
|
|
@ -42,10 +42,10 @@ public class VMUtils
|
|||
throw new UnsupportedOperationException("VM.maxDirectMemory doesn't exist, cannot do memory check.", e);
|
||||
}
|
||||
catch (InvocationTargetException e) {
|
||||
throw new RuntimeException("static method shouldn't throw this", e);
|
||||
throw new UnsupportedOperationException("static method shouldn't throw this", e);
|
||||
}
|
||||
catch (IllegalAccessException e) {
|
||||
throw new RuntimeException("public method, shouldn't throw this", e);
|
||||
throw new UnsupportedOperationException("public method, shouldn't throw this", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,8 +27,8 @@ import com.metamx.common.lifecycle.Lifecycle;
|
|||
import io.druid.common.config.ConfigManager;
|
||||
import io.druid.common.config.ConfigManagerConfig;
|
||||
import io.druid.common.config.JacksonConfigManager;
|
||||
import io.druid.db.MetadataStorageConnector;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.db;
|
||||
package io.druid.metadata;
|
||||
|
||||
/**
|
||||
*/
|
|
@ -17,12 +17,10 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.db;
|
||||
package io.druid.metadata;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MetadataStorageConnectorConfig
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.db;
|
||||
package io.druid.metadata;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
|
@ -40,6 +40,36 @@ See [Examples](Examples.html). This firehose creates a stream of random numbers.
|
|||
|
||||
This firehose ingests events from a define rabbit-mq queue.
|
||||
|
||||
#### LocalFirehose
|
||||
|
||||
This Firehose can be used to read the data from files on local disk.
|
||||
It can be used for POCs to ingest data on disk.
|
||||
A sample local firehose spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "local",
|
||||
"filter" : "*.csv",
|
||||
"parser" : {
|
||||
"timestampSpec": {
|
||||
"column": "mytimestamp",
|
||||
"format": "yyyy-MM-dd HH:mm:ss"
|
||||
},
|
||||
"data": {
|
||||
"format": "csv",
|
||||
"columns": [...],
|
||||
"dimensions": [...]
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "local".|yes|
|
||||
|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) for more information.|yes|
|
||||
|data|A data spec similar to what is used for batch ingestion.|yes|
|
||||
|
||||
#### IngestSegmentFirehose
|
||||
|
||||
This Firehose can be used to read the data from existing druid segments.
|
||||
|
@ -63,11 +93,6 @@ A sample ingest firehose spec is shown below -
|
|||
|metrics|The list of metrics to select. If left empty, no metrics are returned. If left null or not defined, all metrics are selected.|no|
|
||||
|filter| See [Filters](Filters.html)|yes|
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
Parsing Data
|
||||
------------
|
||||
|
||||
|
|
|
@ -42,6 +42,15 @@ You can check `<BROKER_IP>:<PORT>/druid/v2/datasources/<YOUR_DATASOURCE>?interva
|
|||
You can use IngestSegmentFirehose with index task to ingest existing druid segments using a new schema and change the name, dimensions, metrics, rollup, etc. of the segment.
|
||||
See [Firehose](Firehose.html) for more details on IngestSegmentFirehose.
|
||||
|
||||
## How can I change the granularity of existing data in Druid?
|
||||
|
||||
In a lot of situations you may want to lower the granularity of older data. Example, any data older than 1 month has only hour level granularity but newer data has minute level granularity.
|
||||
|
||||
To do this use the IngestSegmentFirehose and run an indexer task. The IngestSegment firehose will allow you to take in existing segments from Druid and aggregate them and feed them back into druid. It will also allow you to filter the data in those segments while feeding it back in. This means if there are rows you want to delete, you can just filter them away during re-ingestion.
|
||||
|
||||
Typically the above will be run as a batch job to say everyday feed in a chunk of data and aggregate it.
|
||||
|
||||
|
||||
## More information
|
||||
|
||||
Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
||||
|
|
|
@ -19,12 +19,13 @@
|
|||
package io.druid.storage.hdfs.tasklog;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.tasklogs.TaskLogs;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -34,71 +35,77 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* Indexer hdfs task logs, to support storing hdfs tasks to hdfs
|
||||
*
|
||||
* Created by Frank Ren on 6/20/14.
|
||||
* Indexer hdfs task logs, to support storing hdfs tasks to hdfs.
|
||||
*/
|
||||
public class HdfsTaskLogs implements TaskLogs
|
||||
{
|
||||
private static final Logger log = new Logger(HdfsTaskLogs.class);
|
||||
private static final Logger log = new Logger(HdfsTaskLogs.class);
|
||||
|
||||
private final HdfsTaskLogsConfig config;
|
||||
private final HdfsTaskLogsConfig config;
|
||||
|
||||
@Inject
|
||||
public HdfsTaskLogs(HdfsTaskLogsConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
@Inject
|
||||
public HdfsTaskLogs(HdfsTaskLogsConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushTaskLog(String taskId, File logFile) throws IOException
|
||||
{
|
||||
final Path path = getTaskLogFileFromId(taskId);
|
||||
log.info("Writing task log to: %s", path);
|
||||
Configuration conf = new Configuration();
|
||||
final FileSystem fs = FileSystem.get(conf);
|
||||
FileUtil.copy(logFile, fs, path, false, conf);
|
||||
log.info("Wrote task log to: %s", path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskId, final long offset) throws IOException
|
||||
{
|
||||
final Path path = getTaskLogFileFromId(taskId);
|
||||
final FileSystem fs = FileSystem.get(new Configuration());
|
||||
if (fs.exists(path)) {
|
||||
return Optional.<InputSupplier<InputStream>>of(
|
||||
new InputSupplier<InputStream>()
|
||||
{
|
||||
@Override
|
||||
public InputStream getInput() throws IOException
|
||||
{
|
||||
log.info("Reading task log from: %s", path);
|
||||
final long seekPos;
|
||||
if (offset < 0) {
|
||||
final FileStatus stat = fs.getFileStatus(path);
|
||||
seekPos = Math.max(0, stat.getLen() + offset);
|
||||
} else {
|
||||
seekPos = offset;
|
||||
}
|
||||
final FSDataInputStream inputStream = fs.open(path);
|
||||
inputStream.seek(seekPos);
|
||||
log.info("Read task log from: %s (seek = %,d)", path, seekPos);
|
||||
return inputStream;
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pushTaskLog(String taskId, File logFile) throws IOException
|
||||
{
|
||||
final Path path = getTaskLogFileFromId(taskId);
|
||||
log.info("writing task log to: %s", path);
|
||||
Configuration conf = new Configuration();
|
||||
final FileSystem fs = FileSystem.get(conf);
|
||||
FileUtil.copy(logFile, fs, path, false, conf);
|
||||
log.info("wrote task log to: %s", path);
|
||||
}
|
||||
/**
|
||||
* Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
|
||||
* path names. So we format paths differently for HDFS.
|
||||
*/
|
||||
private Path getTaskLogFileFromId(String taskId)
|
||||
{
|
||||
return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_")));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<InputSupplier<InputStream>> streamTaskLog(final String taskId, final long offset) throws IOException
|
||||
{
|
||||
final Path path = getTaskLogFileFromId(taskId);
|
||||
final FileSystem fs = FileSystem.get(new Configuration());
|
||||
if (fs.exists(path)) {
|
||||
return Optional.<InputSupplier<InputStream>>of(
|
||||
new InputSupplier<InputStream>() {
|
||||
@Override
|
||||
public InputStream getInput() throws IOException
|
||||
{
|
||||
log.info("reading task log from: %s", path);
|
||||
final InputStream inputStream = fs.open(path);
|
||||
ByteStreams.skipFully(inputStream, offset);
|
||||
log.info("read task log from: %s", path);
|
||||
return inputStream;
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
|
||||
* path names. So we format paths differently for HDFS.
|
||||
*/
|
||||
private Path getTaskLogFileFromId(String taskId)
|
||||
{
|
||||
return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_")));
|
||||
}
|
||||
|
||||
// some hadoop version Path.mergePaths does not exist
|
||||
private static String mergePaths(String path1, String path2)
|
||||
{
|
||||
return path1 + (path1.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR) + path2;
|
||||
}
|
||||
// some hadoop version Path.mergePaths does not exist
|
||||
private static String mergePaths(String path1, String path2)
|
||||
{
|
||||
return path1 + (path1.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR) + path2;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -23,19 +23,23 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import javax.validation.constraints.NotNull;
|
||||
|
||||
/**
|
||||
* Indexer hdfs task logs configuration
|
||||
*
|
||||
* Created by Frank Ren on 6/20/14.
|
||||
* Indexer hdfs task logs configuration.
|
||||
*/
|
||||
public class HdfsTaskLogsConfig
|
||||
{
|
||||
@JsonProperty
|
||||
@NotNull
|
||||
private String directory;
|
||||
|
||||
public String getDirectory()
|
||||
{
|
||||
return directory;
|
||||
}
|
||||
@JsonProperty
|
||||
@NotNull
|
||||
private String directory;
|
||||
|
||||
public HdfsTaskLogsConfig(String directory)
|
||||
{
|
||||
this.directory = directory;
|
||||
}
|
||||
|
||||
public String getDirectory()
|
||||
{
|
||||
return directory;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
package io.druid.indexing.common.tasklogs;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.storage.hdfs.tasklog.HdfsTaskLogs;
|
||||
import io.druid.storage.hdfs.tasklog.HdfsTaskLogsConfig;
|
||||
import io.druid.tasklogs.TaskLogs;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Map;
|
||||
|
||||
public class HdfsTaskLogsTest
|
||||
{
|
||||
@Test
|
||||
public void testSimple() throws Exception
|
||||
{
|
||||
final File tmpDir = Files.createTempDir();
|
||||
try {
|
||||
final File logDir = new File(tmpDir, "logs");
|
||||
final File logFile = new File(tmpDir, "log");
|
||||
Files.write("blah", logFile, Charsets.UTF_8);
|
||||
final TaskLogs taskLogs = new HdfsTaskLogs(new HdfsTaskLogsConfig(logDir.toString()));
|
||||
taskLogs.pushTaskLog("foo", logFile);
|
||||
|
||||
final Map<Long, String> expected = ImmutableMap.of(0L, "blah", 1L, "lah", -2L, "ah", -5L, "blah");
|
||||
for (Map.Entry<Long, String> entry : expected.entrySet()) {
|
||||
final byte[] bytes = ByteStreams.toByteArray(taskLogs.streamTaskLog("foo", entry.getKey()).get().getInput());
|
||||
final String string = new String(bytes);
|
||||
Assert.assertEquals(String.format("Read with offset %,d", entry.getKey()), string, entry.getValue());
|
||||
}
|
||||
}
|
||||
finally {
|
||||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -390,6 +390,11 @@ public class HadoopDruidIndexerConfig
|
|||
}
|
||||
}
|
||||
|
||||
public boolean isPersistInHeap()
|
||||
{
|
||||
return schema.getTuningConfig().isPersistInHeap();
|
||||
}
|
||||
|
||||
/******************************************
|
||||
Path helper logic
|
||||
******************************************/
|
||||
|
|
|
@ -57,7 +57,11 @@ public class HadoopDruidIndexerJob implements Jobby
|
|||
List<Jobby> jobs = Lists.newArrayList();
|
||||
JobHelper.ensurePaths(config);
|
||||
|
||||
indexJob = new IndexGeneratorJob(config);
|
||||
if (config.isPersistInHeap()) {
|
||||
indexJob = new IndexGeneratorJob(config);
|
||||
} else {
|
||||
indexJob = new LegacyIndexGeneratorJob(config);
|
||||
}
|
||||
jobs.add(indexJob);
|
||||
|
||||
if (metadataStorageUpdaterJob != null) {
|
||||
|
@ -66,15 +70,17 @@ public class HadoopDruidIndexerJob implements Jobby
|
|||
log.info("No updaterJobSpec set, not uploading to database");
|
||||
}
|
||||
|
||||
jobs.add(new Jobby()
|
||||
{
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
publishedSegments = IndexGeneratorJob.getPublishedSegments(config);
|
||||
return true;
|
||||
}
|
||||
});
|
||||
jobs.add(
|
||||
new Jobby()
|
||||
{
|
||||
@Override
|
||||
public boolean run()
|
||||
{
|
||||
publishedSegments = IndexGeneratorJob.getPublishedSegments(config);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
||||
JobHelper.runJobs(jobs, config);
|
||||
|
|
|
@ -159,13 +159,15 @@ public class HadoopIngestionSpec extends IngestionSpec<HadoopIOConfig, HadoopTun
|
|||
version,
|
||||
thePartitionSpec,
|
||||
shardSpecs,
|
||||
rollupSpec == null ? 50000 : rollupSpec.rowFlushBoundary,
|
||||
null,
|
||||
leaveIntermediate,
|
||||
cleanupOnFailure,
|
||||
overwriteFiles,
|
||||
ignoreInvalidRows,
|
||||
jobProperties,
|
||||
combineText
|
||||
combineText,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,6 +53,8 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
false,
|
||||
false,
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
@ -68,6 +70,8 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
private final boolean ignoreInvalidRows;
|
||||
private final Map<String, String> jobProperties;
|
||||
private final boolean combineText;
|
||||
private final boolean persistInHeap;
|
||||
private final boolean ingestOffheap;
|
||||
|
||||
@JsonCreator
|
||||
public HadoopTuningConfig(
|
||||
|
@ -81,7 +85,9 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
|
||||
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows,
|
||||
final @JsonProperty("jobProperties") Map<String, String> jobProperties,
|
||||
final @JsonProperty("combineText") boolean combineText
|
||||
final @JsonProperty("combineText") boolean combineText,
|
||||
final @JsonProperty("persistInHeap") boolean persistInHeap,
|
||||
final @JsonProperty("ingestOffheap") boolean ingestOffheap
|
||||
)
|
||||
{
|
||||
this.workingPath = workingPath == null ? null : workingPath;
|
||||
|
@ -97,6 +103,8 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
? ImmutableMap.<String, String>of()
|
||||
: ImmutableMap.copyOf(jobProperties));
|
||||
this.combineText = combineText;
|
||||
this.persistInHeap = persistInHeap;
|
||||
this.ingestOffheap = ingestOffheap;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -165,6 +173,17 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
return combineText;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isPersistInHeap()
|
||||
{
|
||||
return persistInHeap;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isIngestOffheap(){
|
||||
return ingestOffheap;
|
||||
}
|
||||
|
||||
public HadoopTuningConfig withWorkingPath(String path)
|
||||
{
|
||||
return new HadoopTuningConfig(
|
||||
|
@ -178,7 +197,9 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
overwriteFiles,
|
||||
ignoreInvalidRows,
|
||||
jobProperties,
|
||||
combineText
|
||||
combineText,
|
||||
persistInHeap,
|
||||
ingestOffheap
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -195,7 +216,9 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
overwriteFiles,
|
||||
ignoreInvalidRows,
|
||||
jobProperties,
|
||||
combineText
|
||||
combineText,
|
||||
persistInHeap,
|
||||
ingestOffheap
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -212,7 +235,9 @@ public class HadoopTuningConfig implements TuningConfig
|
|||
overwriteFiles,
|
||||
ignoreInvalidRows,
|
||||
jobProperties,
|
||||
combineText
|
||||
combineText,
|
||||
persistInHeap,
|
||||
ingestOffheap
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,13 +35,17 @@ import com.metamx.common.guava.CloseQuietly;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.offheap.OffheapBufferPool;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.IndexMaker;
|
||||
import io.druid.segment.LoggingProgressIndicator;
|
||||
import io.druid.segment.ProgressIndicator;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.SegmentUtils;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.segment.incremental.OffheapIncrementalIndex;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
|
@ -86,20 +90,9 @@ import java.util.zip.ZipOutputStream;
|
|||
public class IndexGeneratorJob implements Jobby
|
||||
{
|
||||
private static final Logger log = new Logger(IndexGeneratorJob.class);
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
private IndexGeneratorStats jobStats;
|
||||
|
||||
public IndexGeneratorJob(
|
||||
HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.jobStats = new IndexGeneratorStats();
|
||||
}
|
||||
|
||||
public static List<DataSegment> getPublishedSegments(HadoopDruidIndexerConfig config)
|
||||
{
|
||||
|
||||
final Configuration conf = new Configuration();
|
||||
final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.jsonMapper;
|
||||
|
||||
|
@ -130,6 +123,22 @@ public class IndexGeneratorJob implements Jobby
|
|||
return publishedSegments;
|
||||
}
|
||||
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
private IndexGeneratorStats jobStats;
|
||||
|
||||
public IndexGeneratorJob(
|
||||
HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.jobStats = new IndexGeneratorStats();
|
||||
}
|
||||
|
||||
protected void setReducerClass(final Job job)
|
||||
{
|
||||
job.setReducerClass(IndexGeneratorReducer.class);
|
||||
}
|
||||
|
||||
public IndexGeneratorStats getJobStats()
|
||||
{
|
||||
return jobStats;
|
||||
|
@ -161,7 +170,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
job.setNumReduceTasks(Iterables.size(config.getAllBuckets().get()));
|
||||
job.setPartitionerClass(IndexGeneratorPartitioner.class);
|
||||
|
||||
job.setReducerClass(IndexGeneratorReducer.class);
|
||||
setReducerClass(job);
|
||||
job.setOutputKeyClass(BytesWritable.class);
|
||||
job.setOutputValueClass(Text.class);
|
||||
job.setOutputFormatClass(IndexGeneratorOutputFormat.class);
|
||||
|
@ -190,7 +199,6 @@ public class IndexGeneratorJob implements Jobby
|
|||
}
|
||||
|
||||
public static class IndexGeneratorMapper extends HadoopDruidIndexerMapper<BytesWritable, Text>
|
||||
|
||||
{
|
||||
@Override
|
||||
protected void innerMap(
|
||||
|
@ -256,6 +264,42 @@ public class IndexGeneratorJob implements Jobby
|
|||
private List<String> metricNames = Lists.newArrayList();
|
||||
private StringInputRowParser parser;
|
||||
|
||||
protected ProgressIndicator makeProgressIndicator(final Context context)
|
||||
{
|
||||
return new LoggingProgressIndicator("IndexGeneratorJob")
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
protected File persist(
|
||||
final IncrementalIndex index,
|
||||
final Interval interval,
|
||||
final File file,
|
||||
final ProgressIndicator progressIndicator
|
||||
) throws IOException
|
||||
{
|
||||
return IndexMaker.persist(
|
||||
index, interval, file, progressIndicator
|
||||
);
|
||||
}
|
||||
|
||||
protected File mergeQueryableIndex(
|
||||
final List<QueryableIndex> indexes,
|
||||
final AggregatorFactory[] aggs,
|
||||
final File file,
|
||||
ProgressIndicator progressIndicator
|
||||
) throws IOException
|
||||
{
|
||||
return IndexMaker.mergeQueryableIndex(
|
||||
indexes, aggs, file, progressIndicator
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
|
@ -282,113 +326,84 @@ public class IndexGeneratorJob implements Jobby
|
|||
final AggregatorFactory[] aggs = config.getSchema().getDataSchema().getAggregators();
|
||||
|
||||
IncrementalIndex index = makeIncrementalIndex(bucket, aggs);
|
||||
try {
|
||||
File baseFlushFile = File.createTempFile("base", "flush");
|
||||
baseFlushFile.delete();
|
||||
baseFlushFile.mkdirs();
|
||||
|
||||
File baseFlushFile = File.createTempFile("base", "flush");
|
||||
baseFlushFile.delete();
|
||||
baseFlushFile.mkdirs();
|
||||
Set<File> toMerge = Sets.newTreeSet();
|
||||
int indexCount = 0;
|
||||
int lineCount = 0;
|
||||
int runningTotalLineCount = 0;
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
Set<File> toMerge = Sets.newTreeSet();
|
||||
int indexCount = 0;
|
||||
int lineCount = 0;
|
||||
int runningTotalLineCount = 0;
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
Set<String> allDimensionNames = Sets.newHashSet();
|
||||
|
||||
for (final Text value : values) {
|
||||
context.progress();
|
||||
final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(value.toString()));
|
||||
allDimensionNames.addAll(inputRow.getDimensions());
|
||||
|
||||
int numRows = index.add(inputRow);
|
||||
++lineCount;
|
||||
|
||||
if (numRows >= config.getSchema().getTuningConfig().getRowFlushBoundary()) {
|
||||
log.info(
|
||||
"%,d lines to %,d rows in %,d millis",
|
||||
lineCount - runningTotalLineCount,
|
||||
numRows,
|
||||
System.currentTimeMillis() - startTime
|
||||
);
|
||||
runningTotalLineCount = lineCount;
|
||||
|
||||
final File file = new File(baseFlushFile, String.format("index%,05d", indexCount));
|
||||
toMerge.add(file);
|
||||
Set<String> allDimensionNames = Sets.newHashSet();
|
||||
final ProgressIndicator progressIndicator = makeProgressIndicator(context);
|
||||
|
||||
for (final Text value : values) {
|
||||
context.progress();
|
||||
IndexMerger.persist(
|
||||
index, interval, file, new IndexMerger.ProgressIndicator()
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
}
|
||||
}
|
||||
);
|
||||
index = makeIncrementalIndex(bucket, aggs);
|
||||
final InputRow inputRow = index.formatRow(parser.parse(value.toString()));
|
||||
allDimensionNames.addAll(inputRow.getDimensions());
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
++indexCount;
|
||||
}
|
||||
}
|
||||
int numRows = index.add(inputRow);
|
||||
++lineCount;
|
||||
|
||||
log.info("%,d lines completed.", lineCount);
|
||||
if (numRows >= config.getSchema().getTuningConfig().getRowFlushBoundary()) {
|
||||
log.info(
|
||||
"%,d lines to %,d rows in %,d millis",
|
||||
lineCount - runningTotalLineCount,
|
||||
numRows,
|
||||
System.currentTimeMillis() - startTime
|
||||
);
|
||||
runningTotalLineCount = lineCount;
|
||||
|
||||
List<QueryableIndex> indexes = Lists.newArrayListWithCapacity(indexCount);
|
||||
final File mergedBase;
|
||||
final File file = new File(baseFlushFile, String.format("index%,05d", indexCount));
|
||||
toMerge.add(file);
|
||||
|
||||
if (toMerge.size() == 0) {
|
||||
if (index.isEmpty()) {
|
||||
throw new IAE("If you try to persist empty indexes you are going to have a bad time");
|
||||
}
|
||||
|
||||
mergedBase = new File(baseFlushFile, "merged");
|
||||
IndexMerger.persist(
|
||||
index, interval, mergedBase, new IndexMerger.ProgressIndicator()
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
persist(index, interval, file, progressIndicator);
|
||||
// close this index and make a new one
|
||||
index.close();
|
||||
index = makeIncrementalIndex(bucket, aggs);
|
||||
|
||||
startTime = System.currentTimeMillis();
|
||||
++indexCount;
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
if (!index.isEmpty()) {
|
||||
final File finalFile = new File(baseFlushFile, "final");
|
||||
IndexMerger.persist(
|
||||
index, interval, finalFile, new IndexMerger.ProgressIndicator()
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
}
|
||||
}
|
||||
);
|
||||
toMerge.add(finalFile);
|
||||
}
|
||||
|
||||
log.info("%,d lines completed.", lineCount);
|
||||
|
||||
List<QueryableIndex> indexes = Lists.newArrayListWithCapacity(indexCount);
|
||||
final File mergedBase;
|
||||
|
||||
if (toMerge.size() == 0) {
|
||||
if (index.isEmpty()) {
|
||||
throw new IAE("If you try to persist empty indexes you are going to have a bad time");
|
||||
}
|
||||
|
||||
mergedBase = new File(baseFlushFile, "merged");
|
||||
persist(index, interval, mergedBase, progressIndicator);
|
||||
} else {
|
||||
if (!index.isEmpty()) {
|
||||
final File finalFile = new File(baseFlushFile, "final");
|
||||
persist(index, interval, finalFile, progressIndicator);
|
||||
toMerge.add(finalFile);
|
||||
}
|
||||
|
||||
for (File file : toMerge) {
|
||||
indexes.add(IndexIO.loadIndex(file));
|
||||
}
|
||||
mergedBase = mergeQueryableIndex(
|
||||
indexes, aggs, new File(baseFlushFile, "merged"), progressIndicator
|
||||
);
|
||||
}
|
||||
serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames));
|
||||
for (File file : toMerge) {
|
||||
indexes.add(IndexIO.loadIndex(file));
|
||||
FileUtils.deleteDirectory(file);
|
||||
}
|
||||
mergedBase = IndexMerger.mergeQueryableIndex(
|
||||
indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator()
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
serializeOutIndex(context, bucket, mergedBase, Lists.newArrayList(allDimensionNames));
|
||||
|
||||
for (File file : toMerge) {
|
||||
FileUtils.deleteDirectory(file);
|
||||
finally {
|
||||
index.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -448,7 +463,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
} else if (outputFS instanceof DistributedFileSystem) {
|
||||
loadSpec = ImmutableMap.<String, Object>of(
|
||||
"type", "hdfs",
|
||||
"path", indexOutURI.getPath()
|
||||
"path", indexOutURI.toString()
|
||||
);
|
||||
} else {
|
||||
throw new ISE("Unknown file system[%s]", outputFS.getClass());
|
||||
|
@ -616,14 +631,29 @@ public class IndexGeneratorJob implements Jobby
|
|||
|
||||
private IncrementalIndex makeIncrementalIndex(Bucket theBucket, AggregatorFactory[] aggs)
|
||||
{
|
||||
return new IncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(theBucket.time.getMillis())
|
||||
.withSpatialDimensions(config.getSchema().getDataSchema().getParser())
|
||||
.withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity())
|
||||
.withMetrics(aggs)
|
||||
.build()
|
||||
);
|
||||
int aggsSize = 0;
|
||||
for (AggregatorFactory agg : aggs) {
|
||||
aggsSize += agg.getMaxIntermediateSize();
|
||||
}
|
||||
final HadoopTuningConfig tuningConfig = config.getSchema().getTuningConfig();
|
||||
int bufferSize = aggsSize * tuningConfig.getRowFlushBoundary();
|
||||
final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(theBucket.time.getMillis())
|
||||
.withDimensionsSpec(config.getSchema().getDataSchema().getParser())
|
||||
.withQueryGranularity(config.getSchema().getDataSchema().getGranularitySpec().getQueryGranularity())
|
||||
.withMetrics(aggs)
|
||||
.build();
|
||||
if (tuningConfig.isIngestOffheap()) {
|
||||
return new OffheapIncrementalIndex(
|
||||
indexSchema,
|
||||
new OffheapBufferPool(bufferSize)
|
||||
);
|
||||
} else {
|
||||
return new IncrementalIndex(
|
||||
indexSchema,
|
||||
new OffheapBufferPool(bufferSize)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void createNewZipEntry(ZipOutputStream out, String name) throws IOException
|
||||
|
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer;
|
||||
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.BaseProgressIndicator;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.ProgressIndicator;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LegacyIndexGeneratorJob extends IndexGeneratorJob
|
||||
{
|
||||
public LegacyIndexGeneratorJob(
|
||||
HadoopDruidIndexerConfig config
|
||||
)
|
||||
{
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setReducerClass(Job job)
|
||||
{
|
||||
job.setReducerClass(LegacyIndexGeneratorReducer.class);
|
||||
}
|
||||
|
||||
public static class LegacyIndexGeneratorReducer extends IndexGeneratorJob.IndexGeneratorReducer
|
||||
{
|
||||
@Override
|
||||
protected ProgressIndicator makeProgressIndicator(final Context context)
|
||||
{
|
||||
return new BaseProgressIndicator()
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
context.progress();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
protected File persist(
|
||||
IncrementalIndex index, Interval interval, File file, ProgressIndicator progressIndicator
|
||||
) throws IOException
|
||||
{
|
||||
return IndexMerger.persist(index, interval, file, progressIndicator);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected File mergeQueryableIndex(
|
||||
List<QueryableIndex> indexes,
|
||||
AggregatorFactory[] aggs,
|
||||
File file,
|
||||
ProgressIndicator progressIndicator
|
||||
) throws IOException
|
||||
{
|
||||
return IndexMerger.mergeQueryableIndex(indexes, aggs, file, progressIndicator);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -30,6 +30,7 @@ import java.util.List;
|
|||
*
|
||||
* Adjust to JsonCreator and final fields when resolved.
|
||||
*/
|
||||
@Deprecated
|
||||
public class DataRollupSpec
|
||||
{
|
||||
@JsonProperty
|
||||
|
@ -38,24 +39,8 @@ public class DataRollupSpec
|
|||
@JsonProperty
|
||||
public QueryGranularity rollupGranularity = QueryGranularity.NONE;
|
||||
|
||||
@JsonProperty
|
||||
public int rowFlushBoundary = 500000;
|
||||
|
||||
public DataRollupSpec() {}
|
||||
|
||||
public DataRollupSpec(List<AggregatorFactory> aggs, QueryGranularity rollupGranularity)
|
||||
{
|
||||
this.aggs = aggs;
|
||||
this.rollupGranularity = rollupGranularity;
|
||||
}
|
||||
|
||||
public List<AggregatorFactory> getAggs()
|
||||
{
|
||||
return aggs;
|
||||
}
|
||||
|
||||
public QueryGranularity getRollupGranularity()
|
||||
{
|
||||
return rollupGranularity;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package io.druid.indexer.updater;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Supplier;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.indexer;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.partitions.RandomPartitionsSpec;
|
||||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
|
|
|
@ -37,13 +37,13 @@ public class IndexingServiceTaskLogsModule implements Module
|
|||
public void configure(Binder binder)
|
||||
{
|
||||
PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(FileTaskLogs.class));
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.logs", FileTaskLogsConfig.class);
|
||||
|
||||
final MapBinder<String, TaskLogs> taskLogBinder = Binders.taskLogsBinder(binder);
|
||||
taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class);
|
||||
taskLogBinder.addBinding("file").to(FileTaskLogs.class).in(LazySingleton.class);
|
||||
binder.bind(NoopTaskLogs.class).in(LazySingleton.class);
|
||||
binder.bind(FileTaskLogs.class).in(LazySingleton.class);
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.logs", FileTaskLogsConfig.class);
|
||||
|
||||
binder.bind(TaskLogPusher.class).to(TaskLogs.class);
|
||||
}
|
||||
|
|
|
@ -30,6 +30,15 @@ public class FileTaskLogsConfig
|
|||
@NotNull
|
||||
private File directory = new File("log");
|
||||
|
||||
public FileTaskLogsConfig()
|
||||
{
|
||||
}
|
||||
|
||||
public FileTaskLogsConfig(File directory)
|
||||
{
|
||||
this.directory = directory;
|
||||
}
|
||||
|
||||
public File getDirectory()
|
||||
{
|
||||
return directory;
|
||||
|
|
|
@ -35,7 +35,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.IndexMaker;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.SegmentUtils;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -166,7 +166,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
|||
}
|
||||
|
||||
fileToUpload = new File(tmpSegmentDir, "merged");
|
||||
IndexMerger.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload);
|
||||
IndexMaker.mergeQueryableIndex(indexes, schema.getAggregators(), fileToUpload);
|
||||
}
|
||||
|
||||
// Map merged segment so we can extract dimensions
|
||||
|
@ -211,8 +211,7 @@ public class YeOldePlumberSchool implements PlumberSchool
|
|||
log.info("Spilling index[%d] with rows[%d] to: %s", indexToPersist.getCount(), rowsToPersist, dirToPersist);
|
||||
|
||||
try {
|
||||
|
||||
IndexMerger.persist(
|
||||
IndexMaker.persist(
|
||||
indexToPersist.getIndex(),
|
||||
dirToPersist
|
||||
);
|
||||
|
|
|
@ -27,7 +27,7 @@ import com.google.common.base.Predicate;
|
|||
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.segment.IndexMaker;
|
||||
import io.druid.segment.IndexableAdapter;
|
||||
import io.druid.segment.QueryableIndexIndexableAdapter;
|
||||
import io.druid.segment.Rowboat;
|
||||
|
@ -106,7 +106,7 @@ public class AppendTask extends MergeTaskBase
|
|||
);
|
||||
}
|
||||
|
||||
return IndexMerger.append(adapters, outDir);
|
||||
return IndexMaker.append(adapters, outDir);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,109 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.TaskToolbox;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.IndexableAdapter;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
public class DeleteTask extends AbstractFixedIntervalTask
|
||||
{
|
||||
private static final Logger log = new Logger(DeleteTask.class);
|
||||
|
||||
@JsonCreator
|
||||
public DeleteTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
super(
|
||||
id != null ? id : String.format(
|
||||
"delete_%s_%s_%s_%s",
|
||||
dataSource,
|
||||
interval.getStart(),
|
||||
interval.getEnd(),
|
||||
new DateTime().toString()
|
||||
),
|
||||
dataSource,
|
||||
Preconditions.checkNotNull(interval, "interval")
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "delete";
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Strategy: Create an empty segment covering the interval to be deleted
|
||||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty);
|
||||
|
||||
// Create DataSegment
|
||||
final DataSegment segment =
|
||||
DataSegment.builder()
|
||||
.dataSource(this.getDataSource())
|
||||
.interval(getInterval())
|
||||
.version(myLock.getVersion())
|
||||
.shardSpec(new NoneShardSpec())
|
||||
.build();
|
||||
|
||||
final File outDir = new File(toolbox.getTaskWorkDir(), segment.getIdentifier());
|
||||
final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir);
|
||||
|
||||
// Upload the segment
|
||||
final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, segment);
|
||||
|
||||
log.info(
|
||||
"Uploaded tombstone segment for[%s] interval[%s] with version[%s]",
|
||||
segment.getDataSource(),
|
||||
segment.getInterval(),
|
||||
segment.getVersion()
|
||||
);
|
||||
|
||||
toolbox.pushSegments(ImmutableList.of(uploadedSegment));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
|
@ -147,7 +147,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
granularitySpec.withQueryGranularity(indexGranularity == null ? QueryGranularity.NONE : indexGranularity)
|
||||
),
|
||||
new IndexIOConfig(firehoseFactory),
|
||||
new IndexTuningConfig(targetPartitionSize, rowFlushBoundary, null)
|
||||
new IndexTuningConfig(targetPartitionSize, 0, null)
|
||||
);
|
||||
}
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -401,7 +401,11 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
version,
|
||||
wrappedDataSegmentPusher,
|
||||
tmpDir
|
||||
).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics);
|
||||
).findPlumber(
|
||||
schema,
|
||||
new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec, null, null),
|
||||
metrics
|
||||
);
|
||||
|
||||
// rowFlushBoundary for this job
|
||||
final int myRowFlushBoundary = rowFlushBoundary > 0
|
||||
|
@ -555,7 +559,7 @@ public class IndexTask extends AbstractFixedIntervalTask
|
|||
@JsonProperty("targetPartitionSize") int targetPartitionSize,
|
||||
@JsonProperty("rowFlushBoundary") int rowFlushBoundary,
|
||||
@JsonProperty("numShards") @Nullable Integer numShards
|
||||
)
|
||||
)
|
||||
{
|
||||
this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize;
|
||||
this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary;
|
||||
|
|
|
@ -28,7 +28,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.IndexMaker;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
||||
|
@ -60,7 +60,7 @@ public class MergeTask extends MergeTaskBase
|
|||
public File merge(final Map<DataSegment, File> segments, final File outDir)
|
||||
throws Exception
|
||||
{
|
||||
return IndexMerger.mergeQueryableIndex(
|
||||
return IndexMaker.mergeQueryableIndex(
|
||||
Lists.transform(
|
||||
ImmutableList.copyOf(segments.values()),
|
||||
new Function<File, QueryableIndex>()
|
||||
|
|
|
@ -143,7 +143,9 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
null,
|
||||
rejectionPolicy == null ? rejectionPolicyFactory : rejectionPolicy,
|
||||
maxPendingPersists,
|
||||
spec.getShardSpec()
|
||||
spec.getShardSpec(),
|
||||
false,
|
||||
false
|
||||
),
|
||||
null, null, null, null
|
||||
);
|
||||
|
|
|
@ -43,7 +43,6 @@ import io.druid.query.QueryRunner;
|
|||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
|
||||
@JsonSubTypes.Type(name = "merge", value = MergeTask.class),
|
||||
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
|
||||
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
||||
@JsonSubTypes.Type(name = "move", value = MoveTask.class),
|
||||
@JsonSubTypes.Type(name = "archive", value = ArchiveTask.class),
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.indexing.common.tasklogs;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.inject.Inject;
|
||||
|
@ -29,7 +28,6 @@ import io.druid.indexing.common.config.FileTaskLogsConfig;
|
|||
import io.druid.tasklogs.TaskLogs;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
|
@ -69,9 +67,7 @@ public class FileTaskLogs implements TaskLogs
|
|||
@Override
|
||||
public InputStream getInput() throws IOException
|
||||
{
|
||||
final InputStream inputStream = new FileInputStream(file);
|
||||
ByteStreams.skipFully(inputStream, offset);
|
||||
return inputStream;
|
||||
return LogUtils.streamFile(file, offset);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -0,0 +1,30 @@
|
|||
package io.druid.indexing.common.tasklogs;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.channels.Channels;
|
||||
|
||||
public class LogUtils
|
||||
{
|
||||
/**
|
||||
* Open a stream to a file.
|
||||
*
|
||||
* @param offset If zero, stream the entire log. If positive, read from this byte position onwards. If negative,
|
||||
* read this many bytes from the end of the file.
|
||||
*
|
||||
* @return input supplier for this log, if available from this provider
|
||||
*/
|
||||
public static InputStream streamFile(final File file, final long offset) throws IOException
|
||||
{
|
||||
final RandomAccessFile raf = new RandomAccessFile(file, "r");
|
||||
final long rafLength = raf.length();
|
||||
if (offset > 0) {
|
||||
raf.seek(offset);
|
||||
} else if (offset < 0 && offset < rafLength) {
|
||||
raf.seek(Math.max(0, rafLength + offset));
|
||||
}
|
||||
return Channels.newInputStream(raf.getChannel());
|
||||
}
|
||||
}
|
|
@ -44,6 +44,7 @@ import io.druid.guice.annotations.Self;
|
|||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.config.TaskConfig;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.common.tasklogs.LogUtils;
|
||||
import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
|
||||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.server.DruidNode;
|
||||
|
@ -391,41 +392,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
@Override
|
||||
public InputStream getInput() throws IOException
|
||||
{
|
||||
final RandomAccessFile raf = new RandomAccessFile(processHolder.logFile, "r");
|
||||
final long rafLength = raf.length();
|
||||
if (offset > 0) {
|
||||
raf.seek(offset);
|
||||
} else if (offset < 0 && offset < rafLength) {
|
||||
raf.seek(Math.max(0, rafLength + offset));
|
||||
}
|
||||
return Channels.newInputStream(raf.getChannel());
|
||||
return LogUtils.streamFile(processHolder.logFile, offset);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private int findUnusedPort()
|
||||
{
|
||||
synchronized (tasks) {
|
||||
int port = config.getStartPort();
|
||||
int maxPortSoFar = -1;
|
||||
|
||||
for (ForkingTaskRunnerWorkItem taskWorkItem : tasks.values()) {
|
||||
if (taskWorkItem.processHolder != null) {
|
||||
if (taskWorkItem.processHolder.port > maxPortSoFar) {
|
||||
maxPortSoFar = taskWorkItem.processHolder.port;
|
||||
}
|
||||
|
||||
if (taskWorkItem.processHolder.port == port) {
|
||||
port = maxPortSoFar + 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return port;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem
|
||||
{
|
||||
private volatile boolean shutdown = false;
|
||||
|
|
|
@ -0,0 +1,69 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A snapshot of a {@link io.druid.indexing.overlord.ZkWorker}
|
||||
*/
|
||||
public class ImmutableZkWorker
|
||||
{
|
||||
private final Worker worker;
|
||||
private final int currCapacityUsed;
|
||||
private final Set<String> availabilityGroups;
|
||||
|
||||
public ImmutableZkWorker(Worker worker, int currCapacityUsed, Set<String> availabilityGroups)
|
||||
{
|
||||
this.worker = worker;
|
||||
this.currCapacityUsed = currCapacityUsed;
|
||||
this.availabilityGroups = ImmutableSet.copyOf(availabilityGroups);
|
||||
}
|
||||
|
||||
public Worker getWorker()
|
||||
{
|
||||
return worker;
|
||||
}
|
||||
|
||||
public int getCurrCapacityUsed()
|
||||
{
|
||||
return currCapacityUsed;
|
||||
}
|
||||
|
||||
public Set<String> getAvailabilityGroups()
|
||||
{
|
||||
return availabilityGroups;
|
||||
}
|
||||
|
||||
public boolean isValidVersion(String minVersion)
|
||||
{
|
||||
return worker.getVersion().compareTo(minVersion) >= 0;
|
||||
}
|
||||
|
||||
public boolean canRunTask(Task task)
|
||||
{
|
||||
return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity()
|
||||
&& !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup()));
|
||||
}
|
||||
}
|
|
@ -32,8 +32,8 @@ import com.google.inject.Inject;
|
|||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.db.MetadataStorageConnector;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.actions.TaskAction;
|
||||
|
|
|
@ -25,14 +25,13 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -49,7 +48,7 @@ import io.druid.curator.cache.PathChildrenCacheFactory;
|
|||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
|
@ -70,10 +69,8 @@ import java.io.InputStream;
|
|||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -88,13 +85,13 @@ import java.util.concurrent.TimeUnit;
|
|||
* creating ephemeral nodes in ZK that workers must remove. Workers announce the statuses of the tasks they are running.
|
||||
* Once a task completes, it is up to the RTR to remove the task status and run any necessary cleanup.
|
||||
* The RemoteTaskRunner is event driven and updates state according to ephemeral node changes in ZK.
|
||||
*
|
||||
* <p/>
|
||||
* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will
|
||||
* fail. The RemoteTaskRunner depends on another component to create additional worker resources.
|
||||
* For example, {@link io.druid.indexing.overlord.scaling.ResourceManagementScheduler} can take care of these duties.
|
||||
*
|
||||
* <p/>
|
||||
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker.
|
||||
*
|
||||
* <p/>
|
||||
* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages.
|
||||
*/
|
||||
public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
||||
|
@ -109,8 +106,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
private final CuratorFramework cf;
|
||||
private final PathChildrenCacheFactory pathChildrenCacheFactory;
|
||||
private final PathChildrenCache workerPathCache;
|
||||
private final Supplier<WorkerSetupData> workerSetupData;
|
||||
private final HttpClient httpClient;
|
||||
private final WorkerSelectStrategy strategy;
|
||||
|
||||
// all workers that exist in ZK
|
||||
private final ConcurrentMap<String, ZkWorker> zkWorkers = new ConcurrentHashMap<>();
|
||||
|
@ -135,8 +132,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
ZkPathsConfig zkPaths,
|
||||
CuratorFramework cf,
|
||||
PathChildrenCacheFactory pathChildrenCacheFactory,
|
||||
Supplier<WorkerSetupData> workerSetupData,
|
||||
HttpClient httpClient
|
||||
HttpClient httpClient,
|
||||
WorkerSelectStrategy strategy
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -145,8 +142,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
this.cf = cf;
|
||||
this.pathChildrenCacheFactory = pathChildrenCacheFactory;
|
||||
this.workerPathCache = pathChildrenCacheFactory.make(cf, zkPaths.getIndexerAnnouncementPath());
|
||||
this.workerSetupData = workerSetupData;
|
||||
this.httpClient = httpClient;
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
|
@ -524,11 +521,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
return true;
|
||||
} else {
|
||||
// Nothing running this task, announce it in ZK for a worker to run it
|
||||
ZkWorker zkWorker = findWorkerForTask(task);
|
||||
if (zkWorker != null) {
|
||||
final Optional<ImmutableZkWorker> immutableZkWorker = strategy.findWorkerForTask(
|
||||
ImmutableMap.copyOf(
|
||||
Maps.transformEntries(
|
||||
zkWorkers,
|
||||
new Maps.EntryTransformer<String, ZkWorker, ImmutableZkWorker>()
|
||||
{
|
||||
@Override
|
||||
public ImmutableZkWorker transformEntry(
|
||||
String key, ZkWorker value
|
||||
)
|
||||
{
|
||||
return value.toImmutable();
|
||||
}
|
||||
}
|
||||
)
|
||||
),
|
||||
task
|
||||
);
|
||||
if (immutableZkWorker.isPresent()) {
|
||||
final ZkWorker zkWorker = zkWorkers.get(immutableZkWorker.get().getWorker().getHost());
|
||||
announceTask(task, zkWorker, taskRunnerWorkItem);
|
||||
return true;
|
||||
} else {
|
||||
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -789,37 +805,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
}
|
||||
}
|
||||
|
||||
private ZkWorker findWorkerForTask(final Task task)
|
||||
{
|
||||
TreeSet<ZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||
new Comparator<ZkWorker>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ZkWorker zkWorker, ZkWorker zkWorker2
|
||||
)
|
||||
{
|
||||
int retVal = Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed());
|
||||
if (retVal == 0) {
|
||||
retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost());
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
sortedWorkers.addAll(zkWorkers.values());
|
||||
final String minWorkerVer = config.getMinWorkerVersion();
|
||||
|
||||
for (ZkWorker zkWorker : sortedWorkers) {
|
||||
if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) {
|
||||
return zkWorker;
|
||||
}
|
||||
}
|
||||
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
||||
return null;
|
||||
}
|
||||
|
||||
private void taskComplete(
|
||||
RemoteTaskRunnerWorkItem taskRunnerWorkItem,
|
||||
ZkWorker zkWorker,
|
||||
|
|
|
@ -20,26 +20,25 @@
|
|||
package io.druid.indexing.overlord;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import io.druid.curator.cache.SimplePathChildrenCacheFactory;
|
||||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
|
||||
/**
|
||||
*/
|
||||
*/
|
||||
public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
||||
{
|
||||
private final CuratorFramework curator;
|
||||
private final RemoteTaskRunnerConfig remoteTaskRunnerConfig;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final Supplier<WorkerSetupData> setupDataWatch;
|
||||
private final HttpClient httpClient;
|
||||
private final WorkerSelectStrategy strategy;
|
||||
|
||||
@Inject
|
||||
public RemoteTaskRunnerFactory(
|
||||
|
@ -47,15 +46,16 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
final RemoteTaskRunnerConfig remoteTaskRunnerConfig,
|
||||
final ZkPathsConfig zkPaths,
|
||||
final ObjectMapper jsonMapper,
|
||||
final Supplier<WorkerSetupData> setupDataWatch,
|
||||
@Global final HttpClient httpClient
|
||||
) {
|
||||
@Global final HttpClient httpClient,
|
||||
final WorkerSelectStrategy strategy
|
||||
)
|
||||
{
|
||||
this.curator = curator;
|
||||
this.remoteTaskRunnerConfig = remoteTaskRunnerConfig;
|
||||
this.zkPaths = zkPaths;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.setupDataWatch = setupDataWatch;
|
||||
this.httpClient = httpClient;
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -70,8 +70,8 @@ public class RemoteTaskRunnerFactory implements TaskRunnerFactory
|
|||
.Builder()
|
||||
.withCompressed(remoteTaskRunnerConfig.isCompressZnodes())
|
||||
.build(),
|
||||
setupDataWatch,
|
||||
httpClient
|
||||
httpClient,
|
||||
strategy
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -158,6 +158,11 @@ public class ZkWorker implements Closeable
|
|||
lastCompletedTaskTime.getAndSet(completedTaskTime);
|
||||
}
|
||||
|
||||
public ImmutableZkWorker toImmutable()
|
||||
{
|
||||
return new ImmutableZkWorker(worker, getCurrCapacityUsed(), getAvailabilityGroups());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.ImmutableZkWorker;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FillCapacityWorkerSelectStrategy implements WorkerSelectStrategy
|
||||
{
|
||||
private final RemoteTaskRunnerConfig config;
|
||||
|
||||
@Inject
|
||||
public FillCapacityWorkerSelectStrategy(RemoteTaskRunnerConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
public Optional<ImmutableZkWorker> findWorkerForTask(
|
||||
final ImmutableMap<String, ImmutableZkWorker> zkWorkers,
|
||||
final Task task
|
||||
)
|
||||
{
|
||||
TreeSet<ImmutableZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||
new Comparator<ImmutableZkWorker>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
ImmutableZkWorker zkWorker, ImmutableZkWorker zkWorker2
|
||||
)
|
||||
{
|
||||
int retVal = Ints.compare(zkWorker2.getCurrCapacityUsed(), zkWorker.getCurrCapacityUsed());
|
||||
if (retVal == 0) {
|
||||
retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost());
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
);
|
||||
sortedWorkers.addAll(zkWorkers.values());
|
||||
final String minWorkerVer = config.getMinWorkerVersion();
|
||||
|
||||
for (ImmutableZkWorker zkWorker : sortedWorkers) {
|
||||
if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) {
|
||||
return Optional.of(zkWorker);
|
||||
}
|
||||
}
|
||||
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.ImmutableZkWorker;
|
||||
import io.druid.indexing.overlord.ZkWorker;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* The {@link io.druid.indexing.overlord.RemoteTaskRunner} uses this class to select a worker to assign tasks to.
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = FillCapacityWorkerSelectStrategy.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "fillCapacity", value = FillCapacityWorkerSelectStrategy.class)
|
||||
})
|
||||
public interface WorkerSelectStrategy
|
||||
{
|
||||
/**
|
||||
* Customizable logic for selecting a worker to run a task.
|
||||
*
|
||||
* @param zkWorkers An immutable map of workers to choose from.
|
||||
* @param task The task to assign.
|
||||
*
|
||||
* @return A {@link io.druid.indexing.overlord.ImmutableZkWorker} to run the task if one is available.
|
||||
*/
|
||||
public Optional<ImmutableZkWorker> findWorkerForTask(
|
||||
final ImmutableMap<String, ImmutableZkWorker> zkWorkers,
|
||||
final Task task
|
||||
);
|
||||
}
|
|
@ -243,51 +243,6 @@ public class TaskSerdeTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteTaskSerde() throws Exception
|
||||
{
|
||||
final DeleteTask task = new DeleteTask(
|
||||
null,
|
||||
"foo",
|
||||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
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 DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals("foo", task.getDataSource());
|
||||
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getInterval(), task2.getInterval());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeleteTaskFromJson() throws Exception
|
||||
{
|
||||
final DeleteTask task = (DeleteTask) jsonMapper.readValue(
|
||||
"{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}",
|
||||
Task.class
|
||||
);
|
||||
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 DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertNotNull(task.getId());
|
||||
Assert.assertEquals("foo", task.getDataSource());
|
||||
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getInterval(), task2.getInterval());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendTaskSerde() throws Exception
|
||||
{
|
||||
|
@ -413,7 +368,7 @@ public class TaskSerdeTest
|
|||
true,
|
||||
null,
|
||||
false,
|
||||
new DataRollupSpec(ImmutableList.<AggregatorFactory>of(), QueryGranularity.NONE),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
ImmutableMap.of("foo", "bar"),
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
package io.druid.indexing.common.tasklogs;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.indexing.common.config.FileTaskLogsConfig;
|
||||
import io.druid.tasklogs.TaskLogs;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Map;
|
||||
|
||||
public class FileTaskLogsTest
|
||||
{
|
||||
@Test
|
||||
public void testSimple() throws Exception
|
||||
{
|
||||
final File tmpDir = Files.createTempDir();
|
||||
try {
|
||||
final File logDir = new File(tmpDir, "logs");
|
||||
final File logFile = new File(tmpDir, "log");
|
||||
Files.write("blah", logFile, Charsets.UTF_8);
|
||||
final TaskLogs taskLogs = new FileTaskLogs(new FileTaskLogsConfig(logDir));
|
||||
taskLogs.pushTaskLog("foo", logFile);
|
||||
|
||||
final Map<Long, String> expected = ImmutableMap.of(0L, "blah", 1L, "lah", -2L, "ah", -5L, "blah");
|
||||
for (Map.Entry<Long, String> entry : expected.entrySet()) {
|
||||
final byte[] bytes = ByteStreams.toByteArray(taskLogs.streamTaskLog("foo", entry.getKey()).get().getInput());
|
||||
final String string = new String(bytes);
|
||||
Assert.assertEquals(String.format("Read with offset %,d", entry.getKey()), string, entry.getValue());
|
||||
}
|
||||
}
|
||||
finally {
|
||||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -39,6 +39,8 @@ import io.druid.indexing.common.TestRealtimeTask;
|
|||
import io.druid.indexing.common.TestUtils;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.common.task.TaskResource;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.indexing.worker.TaskAnnouncement;
|
||||
import io.druid.indexing.worker.Worker;
|
||||
|
@ -367,9 +369,10 @@ public class RemoteTaskRunnerTest
|
|||
|
||||
private void makeRemoteTaskRunner() throws Exception
|
||||
{
|
||||
RemoteTaskRunnerConfig config = new TestRemoteTaskRunnerConfig();
|
||||
remoteTaskRunner = new RemoteTaskRunner(
|
||||
jsonMapper,
|
||||
new TestRemoteTaskRunnerConfig(),
|
||||
config,
|
||||
new ZkPathsConfig()
|
||||
{
|
||||
@Override
|
||||
|
@ -380,8 +383,8 @@ public class RemoteTaskRunnerTest
|
|||
},
|
||||
cf,
|
||||
new SimplePathChildrenCacheFactory.Builder().build(),
|
||||
DSuppliers.of(new AtomicReference<WorkerSetupData>(new WorkerSetupData(0, 1, null, null, null))),
|
||||
null
|
||||
null,
|
||||
new FillCapacityWorkerSelectStrategy(config)
|
||||
);
|
||||
|
||||
remoteTaskRunner.start();
|
||||
|
|
|
@ -42,7 +42,7 @@ import io.druid.data.input.FirehoseFactory;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.db.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
|
|
|
@ -22,9 +22,9 @@ package io.druid.storage.mysql;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.db.SQLMetadataConnector;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.IDBI;
|
||||
|
|
|
@ -23,23 +23,23 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.inject.Binder;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Provides;
|
||||
import io.druid.db.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.db.MetadataRuleManager;
|
||||
import io.druid.db.MetadataRuleManagerProvider;
|
||||
import io.druid.db.MetadataSegmentManager;
|
||||
import io.druid.db.MetadataSegmentManagerProvider;
|
||||
import io.druid.db.MetadataSegmentPublisherProvider;
|
||||
import io.druid.db.MetadataStorageConnector;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.db.SQLMetadataConnector;
|
||||
import io.druid.db.SQLMetadataRuleManager;
|
||||
import io.druid.db.SQLMetadataRuleManagerProvider;
|
||||
import io.druid.db.SQLMetadataSegmentManager;
|
||||
import io.druid.db.SQLMetadataSegmentManagerProvider;
|
||||
import io.druid.db.SQLMetadataSegmentPublisher;
|
||||
import io.druid.db.SQLMetadataSegmentPublisherProvider;
|
||||
import io.druid.db.SQLMetadataStorageActionHandler;
|
||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.metadata.MetadataRuleManager;
|
||||
import io.druid.metadata.MetadataRuleManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentManager;
|
||||
import io.druid.metadata.MetadataSegmentManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentPublisherProvider;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import io.druid.metadata.SQLMetadataRuleManager;
|
||||
import io.druid.metadata.SQLMetadataRuleManagerProvider;
|
||||
import io.druid.metadata.SQLMetadataSegmentManager;
|
||||
import io.druid.metadata.SQLMetadataSegmentManagerProvider;
|
||||
import io.druid.metadata.SQLMetadataSegmentPublisher;
|
||||
import io.druid.metadata.SQLMetadataSegmentPublisherProvider;
|
||||
import io.druid.metadata.SQLMetadataStorageActionHandler;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.PolyBind;
|
||||
|
|
13
pom.xml
13
pom.xml
|
@ -326,22 +326,22 @@
|
|||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-server</artifactId>
|
||||
<version>9.2.2.v20140723</version>
|
||||
<version>9.2.3.v20140905</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-servlet</artifactId>
|
||||
<version>9.2.2.v20140723</version>
|
||||
<version>9.2.3.v20140905</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-servlets</artifactId>
|
||||
<version>9.2.2.v20140723</version>
|
||||
<version>9.2.3.v20140905</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-proxy</artifactId>
|
||||
<version>9.2.2.v20140723</version>
|
||||
<version>9.2.3.v20140905</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
|
@ -431,6 +431,11 @@
|
|||
<version>2.3.0</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mapdb</groupId>
|
||||
<artifactId>mapdb</artifactId>
|
||||
<version>1.0.6</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Test Scope -->
|
||||
<dependency>
|
||||
|
|
|
@ -22,9 +22,9 @@ package io.druid.storage.postgres;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.db.SQLMetadataConnector;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.IDBI;
|
||||
|
|
|
@ -23,23 +23,23 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.inject.Binder;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Provides;
|
||||
import io.druid.db.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.db.MetadataRuleManager;
|
||||
import io.druid.db.MetadataRuleManagerProvider;
|
||||
import io.druid.db.MetadataSegmentManager;
|
||||
import io.druid.db.MetadataSegmentManagerProvider;
|
||||
import io.druid.db.MetadataSegmentPublisherProvider;
|
||||
import io.druid.db.MetadataStorageConnector;
|
||||
import io.druid.db.MetadataStorageConnectorConfig;
|
||||
import io.druid.db.MetadataStorageTablesConfig;
|
||||
import io.druid.db.SQLMetadataConnector;
|
||||
import io.druid.db.SQLMetadataRuleManager;
|
||||
import io.druid.db.SQLMetadataRuleManagerProvider;
|
||||
import io.druid.db.SQLMetadataSegmentManager;
|
||||
import io.druid.db.SQLMetadataSegmentManagerProvider;
|
||||
import io.druid.db.SQLMetadataSegmentPublisher;
|
||||
import io.druid.db.SQLMetadataSegmentPublisherProvider;
|
||||
import io.druid.db.SQLMetadataStorageActionHandler;
|
||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.metadata.MetadataRuleManager;
|
||||
import io.druid.metadata.MetadataRuleManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentManager;
|
||||
import io.druid.metadata.MetadataSegmentManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentPublisherProvider;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import io.druid.metadata.SQLMetadataRuleManager;
|
||||
import io.druid.metadata.SQLMetadataRuleManagerProvider;
|
||||
import io.druid.metadata.SQLMetadataSegmentManager;
|
||||
import io.druid.metadata.SQLMetadataSegmentManagerProvider;
|
||||
import io.druid.metadata.SQLMetadataSegmentPublisher;
|
||||
import io.druid.metadata.SQLMetadataSegmentPublisherProvider;
|
||||
import io.druid.metadata.SQLMetadataStorageActionHandler;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.PolyBind;
|
||||
|
|
|
@ -86,6 +86,10 @@
|
|||
<groupId>net.jpountz.lz4</groupId>
|
||||
<artifactId>lz4</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mapdb</groupId>
|
||||
<artifactId>mapdb</artifactId>
|
||||
</dependency>
|
||||
|
||||
|
||||
<!-- Tests -->
|
||||
|
|
|
@ -19,14 +19,10 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class DataSourceUtil
|
||||
{
|
||||
public static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
|
||||
public static String getMetricName(DataSource dataSource)
|
||||
{
|
||||
final List<String> names = dataSource.getNames();
|
||||
|
|
|
@ -31,15 +31,18 @@ import com.google.common.util.concurrent.ListeningExecutorService;
|
|||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.ResourceClosingSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.query.groupby.GroupByQueryConfig;
|
||||
import io.druid.query.groupby.GroupByQueryHelper;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CancellationException;
|
||||
|
@ -56,11 +59,13 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
private final ListeningExecutorService exec;
|
||||
private final Supplier<GroupByQueryConfig> configSupplier;
|
||||
private final QueryWatcher queryWatcher;
|
||||
private final StupidPool<ByteBuffer> bufferPool;
|
||||
|
||||
public GroupByParallelQueryRunner(
|
||||
ExecutorService exec,
|
||||
Supplier<GroupByQueryConfig> configSupplier,
|
||||
QueryWatcher queryWatcher,
|
||||
StupidPool<ByteBuffer> bufferPool,
|
||||
Iterable<QueryRunner<T>> queryables
|
||||
)
|
||||
{
|
||||
|
@ -68,6 +73,7 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
this.queryWatcher = queryWatcher;
|
||||
this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull()));
|
||||
this.configSupplier = configSupplier;
|
||||
this.bufferPool = bufferPool;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -76,7 +82,8 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
final GroupByQuery query = (GroupByQuery) queryParam;
|
||||
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
|
||||
query,
|
||||
configSupplier.get()
|
||||
configSupplier.get(),
|
||||
bufferPool
|
||||
);
|
||||
final Pair<List, Accumulator<List, T>> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair();
|
||||
final boolean bySegment = query.getContextBySegment(false);
|
||||
|
@ -105,7 +112,8 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
input.run(queryParam, context)
|
||||
.accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs);
|
||||
} else {
|
||||
input.run(queryParam, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
input.run(queryParam, context)
|
||||
.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
}
|
||||
|
||||
return null;
|
||||
|
@ -139,17 +147,21 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
catch (InterruptedException e) {
|
||||
log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId());
|
||||
futures.cancel(true);
|
||||
indexAccumulatorPair.lhs.close();
|
||||
throw new QueryInterruptedException("Query interrupted");
|
||||
}
|
||||
catch (CancellationException e) {
|
||||
indexAccumulatorPair.lhs.close();
|
||||
throw new QueryInterruptedException("Query cancelled");
|
||||
}
|
||||
catch (TimeoutException e) {
|
||||
indexAccumulatorPair.lhs.close();
|
||||
log.info("Query timeout, cancelling pending results for query id [%s]", query.getId());
|
||||
futures.cancel(true);
|
||||
throw new QueryInterruptedException("Query timeout");
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
indexAccumulatorPair.lhs.close();
|
||||
throw Throwables.propagate(e.getCause());
|
||||
}
|
||||
|
||||
|
@ -157,18 +169,20 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
|
|||
return Sequences.simple(bySegmentAccumulatorPair.lhs);
|
||||
}
|
||||
|
||||
return Sequences.simple(
|
||||
Iterables.transform(
|
||||
indexAccumulatorPair.lhs.iterableWithPostAggregations(null),
|
||||
new Function<Row, T>()
|
||||
{
|
||||
@Override
|
||||
public T apply(Row input)
|
||||
{
|
||||
return (T) input;
|
||||
}
|
||||
}
|
||||
)
|
||||
return new ResourceClosingSequence<T>(
|
||||
Sequences.simple(
|
||||
Iterables.transform(
|
||||
indexAccumulatorPair.lhs.iterableWithPostAggregations(null),
|
||||
new Function<Row, T>()
|
||||
{
|
||||
@Override
|
||||
public T apply(Row input)
|
||||
{
|
||||
return (T) input;
|
||||
}
|
||||
}
|
||||
)
|
||||
), indexAccumulatorPair.lhs
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.query;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QueryMetricUtil
|
||||
{
|
||||
public static <T> ServiceMetricEvent.Builder makeQueryTimeMetric(Query<T> query)
|
||||
{
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(query.getType())
|
||||
.setUser5(
|
||||
Lists.transform(
|
||||
query.getIntervals(),
|
||||
new Function<Interval, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(Interval input)
|
||||
{
|
||||
return input.toString();
|
||||
}
|
||||
}
|
||||
).toArray(new String[query.getIntervals().size()])
|
||||
)
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString());
|
||||
}
|
||||
|
||||
public static <T> ServiceMetricEvent.Builder makeRequestTimeMetric(
|
||||
final ObjectMapper jsonMapper, final Query<T> query, final String remoteAddr
|
||||
) throws JsonProcessingException
|
||||
{
|
||||
return makeQueryTimeMetric(query)
|
||||
.setUser3(
|
||||
jsonMapper.writeValueAsString(
|
||||
query.getContext() == null
|
||||
? ImmutableMap.of()
|
||||
: query.getContext()
|
||||
)
|
||||
)
|
||||
.setUser7(remoteAddr)
|
||||
.setUser8(query.getId());
|
||||
}
|
||||
}
|
|
@ -24,20 +24,25 @@ import com.google.common.collect.Lists;
|
|||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.Rows;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.OffheapIncrementalIndex;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
||||
public class GroupByQueryHelper
|
||||
{
|
||||
public static <T> Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> createIndexAccumulatorPair(
|
||||
final GroupByQuery query,
|
||||
final GroupByQueryConfig config
|
||||
final GroupByQueryConfig config,
|
||||
StupidPool<ByteBuffer> bufferPool
|
||||
|
||||
)
|
||||
{
|
||||
final QueryGranularity gran = query.getGranularity();
|
||||
|
@ -69,21 +74,37 @@ public class GroupByQueryHelper
|
|||
}
|
||||
}
|
||||
);
|
||||
IncrementalIndex index = new IncrementalIndex(
|
||||
final IncrementalIndex index;
|
||||
if(query.getContextValue("useOffheap", false)){
|
||||
index = new OffheapIncrementalIndex(
|
||||
// use granularity truncated min timestamp
|
||||
// since incoming truncated timestamps may precede timeStart
|
||||
granTimeStart,
|
||||
gran,
|
||||
aggs.toArray(new AggregatorFactory[aggs.size()]),
|
||||
bufferPool,
|
||||
false
|
||||
);
|
||||
} else {
|
||||
index = new IncrementalIndex(
|
||||
// use granularity truncated min timestamp
|
||||
// since incoming truncated timestamps may precede timeStart
|
||||
granTimeStart,
|
||||
gran,
|
||||
aggs.toArray(new AggregatorFactory[aggs.size()])
|
||||
aggs.toArray(new AggregatorFactory[aggs.size()]),
|
||||
bufferPool,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
Accumulator<IncrementalIndex, T> accumulator = new Accumulator<IncrementalIndex, T>()
|
||||
{
|
||||
@Override
|
||||
public IncrementalIndex accumulate(IncrementalIndex accumulated, T in)
|
||||
{
|
||||
|
||||
if (in instanceof Row) {
|
||||
if (accumulated.add(Rows.toCaseInsensitiveInputRow((Row) in, dimensions), false)
|
||||
if (accumulated.add(Rows.toCaseInsensitiveInputRow((Row) in, dimensions))
|
||||
> config.getMaxResults()) {
|
||||
throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults());
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.query.groupby;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -33,20 +32,23 @@ import com.metamx.common.ISE;
|
|||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.MergeSequence;
|
||||
import com.metamx.common.guava.ResourceClosingSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryDataSource;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.SubqueryQueryRunner;
|
||||
|
@ -58,8 +60,6 @@ import io.druid.query.filter.DimFilter;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
@ -85,19 +85,24 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
);
|
||||
|
||||
private final Supplier<GroupByQueryConfig> configSupplier;
|
||||
|
||||
private final StupidPool<ByteBuffer> bufferPool;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private GroupByQueryEngine engine; // For running the outer query around a subquery
|
||||
|
||||
|
||||
@Inject
|
||||
public GroupByQueryQueryToolChest(
|
||||
Supplier<GroupByQueryConfig> configSupplier,
|
||||
ObjectMapper jsonMapper,
|
||||
GroupByQueryEngine engine
|
||||
GroupByQueryEngine engine,
|
||||
@Global StupidPool<ByteBuffer> bufferPool
|
||||
)
|
||||
{
|
||||
this.configSupplier = configSupplier;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.engine = engine;
|
||||
this.bufferPool = bufferPool;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -123,7 +128,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, Object> context)
|
||||
{
|
||||
// If there's a subquery, merge subquery results and then apply the aggregator
|
||||
|
||||
final DataSource dataSource = query.getDataSource();
|
||||
|
||||
if (dataSource instanceof QueryDataSource) {
|
||||
GroupByQuery subquery;
|
||||
try {
|
||||
|
@ -132,6 +139,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
catch (ClassCastException e) {
|
||||
throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
|
||||
}
|
||||
|
||||
final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, context);
|
||||
final List<AggregatorFactory> aggs = Lists.newArrayList();
|
||||
for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
|
||||
|
@ -148,13 +156,22 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
final GroupByQuery outerQuery = new GroupByQuery.Builder(query)
|
||||
.setLimitSpec(query.getLimitSpec().merge(subquery.getLimitSpec()))
|
||||
.build();
|
||||
IncrementalIndex index = makeIncrementalIndex(innerQuery, subqueryResult);
|
||||
|
||||
final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(
|
||||
makeIncrementalIndex(innerQuery, subqueryResult)
|
||||
return new ResourceClosingSequence<>(
|
||||
outerQuery.applyLimit(
|
||||
engine.process(
|
||||
outerQuery,
|
||||
new IncrementalIndexStorageAdapter(
|
||||
index
|
||||
)
|
||||
)
|
||||
),
|
||||
index
|
||||
);
|
||||
return outerQuery.applyLimit(engine.process(outerQuery, adapter));
|
||||
} else {
|
||||
return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query, context))));
|
||||
final IncrementalIndex index = makeIncrementalIndex(query, runner.run(query, context));
|
||||
return new ResourceClosingSequence<>(query.applyLimit(postAggregate(query, index)), index);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -183,13 +200,13 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
final GroupByQueryConfig config = configSupplier.get();
|
||||
Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
|
||||
query,
|
||||
config
|
||||
config,
|
||||
bufferPool
|
||||
);
|
||||
|
||||
return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Sequence<Row> mergeSequences(Sequence<Sequence<Row>> seqOfSequences)
|
||||
{
|
||||
|
@ -210,19 +227,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(GroupByQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser3(String.format("%,d dims", query.getDimensions().size()))
|
||||
.setUser4("groupBy")
|
||||
.setUser5(Joiner.on(",").join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser3(String.format("%,d dims", query.getDimensions().size()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.query.groupby;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
@ -30,11 +29,12 @@ import com.google.inject.Inject;
|
|||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.ExecutorExecutingSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.query.AbstractPrioritizedCallable;
|
||||
import io.druid.query.ConcatQueryRunner;
|
||||
import io.druid.query.GroupByParallelQueryRunner;
|
||||
|
@ -48,9 +48,9 @@ import io.druid.segment.Segment;
|
|||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -61,25 +61,27 @@ import java.util.concurrent.TimeoutException;
|
|||
*/
|
||||
public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupByQuery>
|
||||
{
|
||||
private static final Logger log = new Logger(GroupByQueryRunnerFactory.class);
|
||||
private final GroupByQueryEngine engine;
|
||||
private final QueryWatcher queryWatcher;
|
||||
private final Supplier<GroupByQueryConfig> config;
|
||||
private final GroupByQueryQueryToolChest toolChest;
|
||||
|
||||
private static final Logger log = new Logger(GroupByQueryRunnerFactory.class);
|
||||
private final StupidPool<ByteBuffer> computationBufferPool;
|
||||
|
||||
@Inject
|
||||
public GroupByQueryRunnerFactory(
|
||||
GroupByQueryEngine engine,
|
||||
QueryWatcher queryWatcher,
|
||||
Supplier<GroupByQueryConfig> config,
|
||||
GroupByQueryQueryToolChest toolChest
|
||||
GroupByQueryQueryToolChest toolChest,
|
||||
@Global StupidPool<ByteBuffer> computationBufferPool
|
||||
)
|
||||
{
|
||||
this.engine = engine;
|
||||
this.queryWatcher = queryWatcher;
|
||||
this.config = config;
|
||||
this.toolChest = toolChest;
|
||||
this.computationBufferPool = computationBufferPool;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -112,7 +114,8 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
|||
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper
|
||||
.createIndexAccumulatorPair(
|
||||
queryParam,
|
||||
config.get()
|
||||
config.get(),
|
||||
computationBufferPool
|
||||
);
|
||||
final Pair<List, Accumulator<List, Row>> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair();
|
||||
final int priority = query.getContextPriority(0);
|
||||
|
@ -131,7 +134,8 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
|||
bySegmentAccumulatorPair.rhs
|
||||
);
|
||||
} else {
|
||||
input.run(query, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
input.run(query, context)
|
||||
.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
}
|
||||
|
||||
return null;
|
||||
|
@ -176,7 +180,8 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
|||
)
|
||||
);
|
||||
} else {
|
||||
return new GroupByParallelQueryRunner(queryExecutor, config, queryWatcher, queryRunners);
|
||||
|
||||
return new GroupByParallelQueryRunner(queryExecutor, config, queryWatcher, computationBufferPool, queryRunners);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,12 +22,10 @@ package io.druid.query.metadata;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.MergeSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
|
@ -36,9 +34,8 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
|
@ -47,7 +44,6 @@ import io.druid.query.metadata.metadata.ColumnAnalysis;
|
|||
import io.druid.query.metadata.metadata.SegmentAnalysis;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -62,14 +58,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
};
|
||||
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
|
||||
|
||||
private final QueryConfig config;
|
||||
|
||||
@Inject
|
||||
public SegmentMetadataQueryQueryToolChest(QueryConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<SegmentAnalysis> mergeResults(final QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
|
@ -158,17 +146,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SegmentMetadataQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(query.getType())
|
||||
.setUser5(Joiner.on(",").join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -195,9 +173,9 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
{
|
||||
byte[] includerBytes = query.getToInclude().getCacheKey();
|
||||
return ByteBuffer.allocate(1 + includerBytes.length)
|
||||
.put(SEGMENT_METADATA_CACHE_PREFIX)
|
||||
.put(includerBytes)
|
||||
.array();
|
||||
.put(SEGMENT_METADATA_CACHE_PREFIX)
|
||||
.put(includerBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -39,9 +38,9 @@ import com.metamx.common.guava.nary.BinaryFn;
|
|||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -53,8 +52,6 @@ import io.druid.query.search.search.SearchHit;
|
|||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -67,7 +64,6 @@ import java.util.Set;
|
|||
public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResultValue>, SearchQuery>
|
||||
{
|
||||
private static final byte SEARCH_QUERY = 0x2;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Result<SearchResultValue>> TYPE_REFERENCE = new TypeReference<Result<SearchResultValue>>()
|
||||
{
|
||||
};
|
||||
|
@ -124,17 +120,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SearchQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("search")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,7 +167,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
|||
final ByteBuffer queryCacheKey = ByteBuffer
|
||||
.allocate(
|
||||
1 + 4 + granularityBytes.length + filterBytes.length +
|
||||
querySpecBytes.length + dimensionsBytesSize
|
||||
querySpecBytes.length + dimensionsBytesSize
|
||||
)
|
||||
.put(SEARCH_QUERY)
|
||||
.put(Ints.toByteArray(query.getLimit()))
|
||||
|
|
|
@ -165,13 +165,16 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
while (!cursor.isDone()) {
|
||||
for (Map.Entry<String, DimensionSelector> entry : dimSelectors.entrySet()) {
|
||||
final DimensionSelector selector = entry.getValue();
|
||||
final IndexedInts vals = selector.getRow();
|
||||
for (int i = 0; i < vals.size(); ++i) {
|
||||
final String dimVal = selector.lookupName(vals.get(i));
|
||||
if (searchQuerySpec.accept(dimVal)) {
|
||||
set.add(new SearchHit(entry.getKey(), dimVal));
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
|
||||
if (selector != null) {
|
||||
final IndexedInts vals = selector.getRow();
|
||||
for (int i = 0; i < vals.size(); ++i) {
|
||||
final String dimVal = selector.lookupName(vals.get(i));
|
||||
if (searchQuerySpec.accept(dimVal)) {
|
||||
set.add(new SearchHit(entry.getKey(), dimVal));
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
|
@ -35,10 +34,10 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -47,8 +46,6 @@ import io.druid.query.ResultMergeQueryRunner;
|
|||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
|
@ -62,7 +59,6 @@ import java.util.Set;
|
|||
public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResultValue>, SelectQuery>
|
||||
{
|
||||
private static final byte SELECT_QUERY = 0x13;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
|
||||
new TypeReference<Object>()
|
||||
{
|
||||
|
@ -126,17 +122,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(SelectQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("Select")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -261,13 +247,13 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
|
|||
new SelectResultValue(
|
||||
(Map<String, Integer>) jsonMapper.convertValue(
|
||||
resultIter.next(), new TypeReference<Map<String, Integer>>()
|
||||
{
|
||||
}
|
||||
{
|
||||
}
|
||||
),
|
||||
(List<EventHolder>) jsonMapper.convertValue(
|
||||
resultIter.next(), new TypeReference<List<EventHolder>>()
|
||||
{
|
||||
}
|
||||
{
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.timeseries;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
@ -33,11 +32,11 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryConfig;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -48,8 +47,6 @@ import io.druid.query.aggregation.MetricManipulationFn;
|
|||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -62,7 +59,6 @@ import java.util.Map;
|
|||
public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
|
||||
{
|
||||
private static final byte TIMESERIES_QUERY = 0x0;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE =
|
||||
new TypeReference<Object>()
|
||||
{
|
||||
|
@ -124,18 +120,8 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(TimeseriesQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4("timeseries")
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.topn;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -37,10 +36,10 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
|||
import io.druid.collections.OrderedMergeSequence;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.DataSourceUtil;
|
||||
import io.druid.query.IntervalChunkingQueryRunner;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryCacheHelper;
|
||||
import io.druid.query.QueryMetricUtil;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
|
@ -52,8 +51,6 @@ import io.druid.query.aggregation.MetricManipulationFn;
|
|||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
@ -65,7 +62,6 @@ import java.util.Map;
|
|||
public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultValue>, TopNQuery>
|
||||
{
|
||||
private static final byte TOPN_QUERY = 0x1;
|
||||
private static final Joiner COMMA_JOIN = Joiner.on(",");
|
||||
private static final TypeReference<Result<TopNResultValue>> TYPE_REFERENCE = new TypeReference<Result<TopNResultValue>>()
|
||||
{
|
||||
};
|
||||
|
@ -139,18 +135,15 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(TopNQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(DataSourceUtil.getMetricName(query.getDataSource()))
|
||||
.setUser4(String.format("topN/%s/%s", query.getThreshold(), query.getDimensionSpec().getDimension()))
|
||||
.setUser5(COMMA_JOIN.join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
return QueryMetricUtil.makeQueryTimeMetric(query)
|
||||
.setUser4(
|
||||
String.format(
|
||||
"topN/%s/%s",
|
||||
query.getThreshold(),
|
||||
query.getDimensionSpec().getDimension()
|
||||
)
|
||||
)
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BaseProgressIndicator implements ProgressIndicator
|
||||
{
|
||||
@Override
|
||||
public void progress()
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startSection(String section)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void progressSection(String section, String message)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopSection(String section)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
}
|
|
@ -68,6 +68,6 @@ public class IncrementalIndexSegment implements Segment
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
// do nothing
|
||||
index.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -228,7 +228,7 @@ public class IndexIO
|
|||
case 6:
|
||||
case 7:
|
||||
log.info("Old version, re-persisting.");
|
||||
IndexMerger.append(
|
||||
IndexMaker.append(
|
||||
Arrays.<IndexableAdapter>asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))),
|
||||
converted
|
||||
);
|
||||
|
@ -718,7 +718,7 @@ public class IndexIO
|
|||
|
||||
return new SimpleQueryableIndex(
|
||||
index.getDataInterval(),
|
||||
new ArrayIndexed<String>(cols, String.class),
|
||||
new ArrayIndexed<>(cols, String.class),
|
||||
index.getAvailableDimensions(),
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.LONG)
|
||||
|
@ -752,8 +752,6 @@ public class IndexIO
|
|||
|
||||
Map<String, Column> columns = Maps.newHashMap();
|
||||
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
for (String columnName : cols) {
|
||||
columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName)));
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -50,6 +50,9 @@ import io.druid.common.utils.JodaUtils;
|
|||
import io.druid.common.utils.SerializerUtils;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.ByteBufferWriter;
|
||||
import io.druid.segment.data.CompressedLongsSupplierSerializer;
|
||||
import io.druid.segment.data.CompressedObjectStrategy;
|
||||
|
@ -114,11 +117,12 @@ public class IndexMerger
|
|||
* @param outDir the directory to persist the data to
|
||||
*
|
||||
* @return the index output directory
|
||||
*
|
||||
* @throws java.io.IOException if an IO error occurs persisting the index
|
||||
*/
|
||||
public static File persist(final IncrementalIndex index, final Interval dataInterval, File outDir) throws IOException
|
||||
{
|
||||
return persist(index, dataInterval, outDir, new NoopProgressIndicator());
|
||||
return persist(index, dataInterval, outDir, new BaseProgressIndicator());
|
||||
}
|
||||
|
||||
public static File persist(
|
||||
|
@ -160,7 +164,7 @@ public class IndexMerger
|
|||
List<QueryableIndex> indexes, final AggregatorFactory[] metricAggs, File outDir
|
||||
) throws IOException
|
||||
{
|
||||
return mergeQueryableIndex(indexes, metricAggs, outDir, new NoopProgressIndicator());
|
||||
return mergeQueryableIndex(indexes, metricAggs, outDir, new BaseProgressIndicator());
|
||||
}
|
||||
|
||||
public static File mergeQueryableIndex(
|
||||
|
@ -189,7 +193,7 @@ public class IndexMerger
|
|||
List<IndexableAdapter> indexes, final AggregatorFactory[] metricAggs, File outDir
|
||||
) throws IOException
|
||||
{
|
||||
return merge(indexes, metricAggs, outDir, new NoopProgressIndicator());
|
||||
return merge(indexes, metricAggs, outDir, new BaseProgressIndicator());
|
||||
}
|
||||
|
||||
public static File merge(
|
||||
|
@ -215,7 +219,7 @@ public class IndexMerger
|
|||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getAvailableDimensions(),
|
||||
input.getDimensionNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
|
@ -241,7 +245,7 @@ public class IndexMerger
|
|||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getAvailableMetrics(),
|
||||
input.getMetricNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
|
@ -312,7 +316,7 @@ public class IndexMerger
|
|||
List<IndexableAdapter> indexes, File outDir
|
||||
) throws IOException
|
||||
{
|
||||
return append(indexes, outDir, new NoopProgressIndicator());
|
||||
return append(indexes, outDir, new BaseProgressIndicator());
|
||||
}
|
||||
|
||||
public static File append(
|
||||
|
@ -333,7 +337,7 @@ public class IndexMerger
|
|||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getAvailableDimensions(),
|
||||
input.getDimensionNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
|
@ -356,7 +360,7 @@ public class IndexMerger
|
|||
public Iterable<String> apply(@Nullable IndexableAdapter input)
|
||||
{
|
||||
return Iterables.transform(
|
||||
input.getAvailableMetrics(),
|
||||
input.getMetricNames(),
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
|
@ -397,12 +401,34 @@ public class IndexMerger
|
|||
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn
|
||||
) throws IOException
|
||||
{
|
||||
Map<String, String> metricTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
|
||||
final Map<String, ColumnCapabilitiesImpl> columnCapabilities = Maps.newHashMap();
|
||||
|
||||
for (IndexableAdapter adapter : indexes) {
|
||||
for (String metric : adapter.getAvailableMetrics()) {
|
||||
metricTypes.put(metric, adapter.getMetricType(metric));
|
||||
for (String dimension : adapter.getDimensionNames()) {
|
||||
ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(dimension);
|
||||
ColumnCapabilities capabilities = adapter.getCapabilities(dimension);
|
||||
if (mergedCapabilities == null) {
|
||||
mergedCapabilities = new ColumnCapabilitiesImpl();
|
||||
mergedCapabilities.setType(ValueType.STRING);
|
||||
}
|
||||
columnCapabilities.put(dimension, mergedCapabilities.merge(capabilities));
|
||||
}
|
||||
for (String metric : adapter.getMetricNames()) {
|
||||
ColumnCapabilitiesImpl mergedCapabilities = columnCapabilities.get(metric);
|
||||
ColumnCapabilities capabilities = adapter.getCapabilities(metric);
|
||||
if (mergedCapabilities == null) {
|
||||
mergedCapabilities = new ColumnCapabilitiesImpl();
|
||||
}
|
||||
columnCapabilities.put(metric, mergedCapabilities.merge(capabilities));
|
||||
|
||||
valueTypes.put(metric, capabilities.getType());
|
||||
metricTypeNames.put(metric, adapter.getMetricType(metric));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
final Interval dataInterval;
|
||||
File v8OutDir = new File(outDir, "v8-tmp");
|
||||
v8OutDir.mkdirs();
|
||||
|
@ -541,14 +567,14 @@ public class IndexMerger
|
|||
|
||||
final int[] dimLookup = new int[mergedDimensions.size()];
|
||||
int count = 0;
|
||||
for (String dim : adapter.getAvailableDimensions()) {
|
||||
for (String dim : adapter.getDimensionNames()) {
|
||||
dimLookup[count] = mergedDimensions.indexOf(dim.toLowerCase());
|
||||
count++;
|
||||
}
|
||||
|
||||
final int[] metricLookup = new int[mergedMetrics.size()];
|
||||
count = 0;
|
||||
for (String metric : adapter.getAvailableMetrics()) {
|
||||
for (String metric : adapter.getMetricNames()) {
|
||||
metricLookup[count] = mergedMetrics.indexOf(metric);
|
||||
count++;
|
||||
}
|
||||
|
@ -580,8 +606,7 @@ public class IndexMerger
|
|||
input.getTimestamp(),
|
||||
newDims,
|
||||
newMetrics,
|
||||
input.getRowNum(),
|
||||
input.getDescriptions()
|
||||
input.getRowNum()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -594,8 +619,7 @@ public class IndexMerger
|
|||
Iterable<Rowboat> theRows = rowMergerFn.apply(boats);
|
||||
|
||||
CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create(
|
||||
ioPeon, "little_end_time", IndexIO.BYTE_ORDER,
|
||||
CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
|
||||
ioPeon, "little_end_time", IndexIO.BYTE_ORDER, CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
|
||||
);
|
||||
|
||||
timeWriter.open();
|
||||
|
@ -608,21 +632,27 @@ public class IndexMerger
|
|||
}
|
||||
|
||||
ArrayList<MetricColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
|
||||
for (Map.Entry<String, String> entry : metricTypes.entrySet()) {
|
||||
String metric = entry.getKey();
|
||||
String typeName = entry.getValue();
|
||||
if ("float".equals(typeName)) {
|
||||
metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon));
|
||||
} else {
|
||||
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
for (String metric : mergedMetrics) {
|
||||
ValueType type = valueTypes.get(metric);
|
||||
switch (type) {
|
||||
case FLOAT:
|
||||
metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon));
|
||||
break;
|
||||
case COMPLEX:
|
||||
final String typeName = metricTypeNames.get(metric);
|
||||
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
|
||||
if (serde == null) {
|
||||
throw new ISE("Unknown type[%s]", typeName);
|
||||
}
|
||||
if (serde == null) {
|
||||
throw new ISE("Unknown type[%s]", typeName);
|
||||
}
|
||||
|
||||
metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde));
|
||||
metWriters.add(new ComplexMetricColumnSerializer(metric, v8OutDir, ioPeon, serde));
|
||||
break;
|
||||
default:
|
||||
throw new ISE("Unknown type[%s]", type);
|
||||
}
|
||||
}
|
||||
|
||||
for (MetricColumnSerializer metWriter : metWriters) {
|
||||
metWriter.open();
|
||||
}
|
||||
|
@ -636,7 +666,6 @@ public class IndexMerger
|
|||
rowNumConversions.add(IntBuffer.wrap(arr));
|
||||
}
|
||||
|
||||
final Map<String, String> descriptions = Maps.newHashMap();
|
||||
for (Rowboat theRow : theRows) {
|
||||
progress.progress();
|
||||
timeWriter.add(theRow.getTimestamp());
|
||||
|
@ -671,8 +700,6 @@ public class IndexMerger
|
|||
);
|
||||
time = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
descriptions.putAll(theRow.getDescriptions());
|
||||
}
|
||||
|
||||
for (IntBuffer rowNumConversion : rowNumConversions) {
|
||||
|
@ -731,7 +758,7 @@ public class IndexMerger
|
|||
);
|
||||
writer.open();
|
||||
|
||||
boolean isSpatialDim = "spatial".equals(descriptions.get(dimension));
|
||||
boolean isSpatialDim = columnCapabilities.get(dimension).hasSpatialIndexes();
|
||||
ByteBufferWriter<ImmutableRTree> spatialWriter = null;
|
||||
RTree tree = null;
|
||||
IOPeon spatialIoPeon = new TmpFileIOPeon();
|
||||
|
@ -1069,8 +1096,7 @@ public class IndexMerger
|
|||
input.getTimestamp(),
|
||||
newDims,
|
||||
input.getMetrics(),
|
||||
input.getRowNum(),
|
||||
input.getDescriptions()
|
||||
input.getRowNum()
|
||||
);
|
||||
|
||||
retVal.addRow(indexNumber, input.getRowNum());
|
||||
|
@ -1150,8 +1176,7 @@ public class IndexMerger
|
|||
lhs.getTimestamp(),
|
||||
lhs.getDims(),
|
||||
metrics,
|
||||
lhs.getRowNum(),
|
||||
lhs.getDescriptions()
|
||||
lhs.getRowNum()
|
||||
);
|
||||
|
||||
for (Rowboat rowboat : Arrays.asList(lhs, rhs)) {
|
||||
|
@ -1165,15 +1190,4 @@ public class IndexMerger
|
|||
return retVal;
|
||||
}
|
||||
}
|
||||
|
||||
public static interface ProgressIndicator
|
||||
{
|
||||
public void progress();
|
||||
}
|
||||
|
||||
private static class NoopProgressIndicator implements ProgressIndicator
|
||||
{
|
||||
@Override
|
||||
public void progress() {}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,14 @@
|
|||
|
||||
package io.druid.segment;
|
||||
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
* An adapter to an index
|
||||
*/
|
||||
public interface IndexableAdapter
|
||||
{
|
||||
|
@ -31,9 +34,9 @@ public interface IndexableAdapter
|
|||
|
||||
int getNumRows();
|
||||
|
||||
Indexed<String> getAvailableDimensions();
|
||||
Indexed<String> getDimensionNames();
|
||||
|
||||
Indexed<String> getAvailableMetrics();
|
||||
Indexed<String> getMetricNames();
|
||||
|
||||
Indexed<String> getDimValueLookup(String dimension);
|
||||
|
||||
|
@ -42,4 +45,6 @@ public interface IndexableAdapter
|
|||
IndexedInts getInverteds(String dimension, String value);
|
||||
|
||||
String getMetricType(String metric);
|
||||
|
||||
ColumnCapabilities getCapabilities(String column);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LoggingProgressIndicator extends BaseProgressIndicator
|
||||
{
|
||||
private static Logger log = new Logger(LoggingProgressIndicator.class);
|
||||
|
||||
private final String progressName;
|
||||
private final Stopwatch global;
|
||||
|
||||
private final Map<String, Stopwatch> sections = Maps.newHashMap();
|
||||
|
||||
public LoggingProgressIndicator(String progressName)
|
||||
{
|
||||
this.progressName = progressName;
|
||||
this.global = Stopwatch.createUnstarted();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
log.info("Starting [%s]", progressName);
|
||||
global.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
long time = global.elapsed(TimeUnit.MILLISECONDS);
|
||||
global.stop();
|
||||
|
||||
log.info("[%s] complete. Elapsed time: [%,d] millis", progressName, time);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startSection(String section)
|
||||
{
|
||||
log.info("[%s]: Starting [%s]", progressName, section);
|
||||
|
||||
Stopwatch sectionWatch = sections.get(section);
|
||||
if (sectionWatch != null) {
|
||||
throw new ISE("[%s]: Cannot start progress tracker for [%s]. It is already started.", progressName, section);
|
||||
}
|
||||
sectionWatch = Stopwatch.createStarted();
|
||||
sections.put(section, sectionWatch);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void progressSection(String section, String message)
|
||||
{
|
||||
Stopwatch sectionWatch = sections.get(section);
|
||||
if (sectionWatch == null) {
|
||||
throw new ISE("[%s]: Cannot progress tracker for [%s]. Nothing started.", progressName, section);
|
||||
}
|
||||
long time = sectionWatch.elapsed(TimeUnit.MILLISECONDS);
|
||||
log.info("[%s]: [%s] : %s. Elapsed time: [%,d] millis", progressName, section, message, time);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopSection(String section)
|
||||
{
|
||||
Stopwatch sectionWatch = sections.remove(section);
|
||||
if (sectionWatch == null) {
|
||||
throw new ISE("[%s]: Cannot stop progress tracker for [%s]. Nothing started.", progressName, section);
|
||||
}
|
||||
long time = sectionWatch.elapsed(TimeUnit.MILLISECONDS);
|
||||
sectionWatch.stop();
|
||||
|
||||
log.info("[%s]: [%s] has completed. Elapsed time: [%,d] millis", progressName, section, time);
|
||||
}
|
||||
}
|
|
@ -1,197 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.segment.data.ConciseCompressedIndexedInts;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedFloats;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.IndexedLongs;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MMappedIndexAdapter implements IndexableAdapter
|
||||
{
|
||||
private final MMappedIndex index;
|
||||
private final int numRows;
|
||||
|
||||
public MMappedIndexAdapter(MMappedIndex index)
|
||||
{
|
||||
this.index = index;
|
||||
|
||||
numRows = index.getReadOnlyTimestamps().size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Interval getDataInterval()
|
||||
{
|
||||
return index.getDataInterval();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
return numRows;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableDimensions()
|
||||
{
|
||||
return index.getAvailableDimensions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableMetrics()
|
||||
{
|
||||
return index.getAvailableMetrics();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getDimValueLookup(String dimension)
|
||||
{
|
||||
return index.getDimValueLookup(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Rowboat> getRows()
|
||||
{
|
||||
return new Iterable<Rowboat>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<Rowboat> iterator()
|
||||
{
|
||||
return new Iterator<Rowboat>()
|
||||
{
|
||||
final IndexedLongs timestamps = index.getReadOnlyTimestamps();
|
||||
final MetricHolder[] metrics;
|
||||
final IndexedFloats[] floatMetrics;
|
||||
final Map<String, Indexed<? extends IndexedInts>> dimensions;
|
||||
|
||||
final int numMetrics = index.getAvailableMetrics().size();
|
||||
|
||||
int currRow = 0;
|
||||
boolean done = false;
|
||||
|
||||
{
|
||||
dimensions = Maps.newLinkedHashMap();
|
||||
for (String dim : index.getAvailableDimensions()) {
|
||||
dimensions.put(dim, index.getDimColumn(dim));
|
||||
}
|
||||
|
||||
final Indexed<String> availableMetrics = index.getAvailableMetrics();
|
||||
metrics = new MetricHolder[availableMetrics.size()];
|
||||
floatMetrics = new IndexedFloats[availableMetrics.size()];
|
||||
for (int i = 0; i < metrics.length; ++i) {
|
||||
metrics[i] = index.getMetricHolder(availableMetrics.get(i));
|
||||
if (metrics[i].getType() == MetricHolder.MetricType.FLOAT) {
|
||||
floatMetrics[i] = metrics[i].getFloatType();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
final boolean hasNext = currRow < numRows;
|
||||
if (!hasNext && !done) {
|
||||
CloseQuietly.close(timestamps);
|
||||
for (IndexedFloats floatMetric : floatMetrics) {
|
||||
CloseQuietly.close(floatMetric);
|
||||
}
|
||||
done = true;
|
||||
}
|
||||
return hasNext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Rowboat next()
|
||||
{
|
||||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
|
||||
int[][] dims = new int[dimensions.size()][];
|
||||
int dimIndex = 0;
|
||||
for (String dim : dimensions.keySet()) {
|
||||
IndexedInts dimVals = dimensions.get(dim).get(currRow);
|
||||
|
||||
int[] theVals = new int[dimVals.size()];
|
||||
for (int j = 0; j < theVals.length; ++j) {
|
||||
theVals[j] = dimVals.get(j);
|
||||
}
|
||||
|
||||
dims[dimIndex++] = theVals;
|
||||
}
|
||||
|
||||
Object[] metricArray = new Object[numMetrics];
|
||||
for (int i = 0; i < metricArray.length; ++i) {
|
||||
switch (metrics[i].getType()) {
|
||||
case FLOAT:
|
||||
metricArray[i] = floatMetrics[i].get(currRow);
|
||||
break;
|
||||
case COMPLEX:
|
||||
metricArray[i] = metrics[i].getComplexType().get(currRow);
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, String> descriptions = Maps.newHashMap();
|
||||
for (String spatialDim : index.getSpatialIndexes().keySet()) {
|
||||
descriptions.put(spatialDim, "spatial");
|
||||
}
|
||||
final Rowboat retVal = new Rowboat(timestamps.get(currRow), dims, metricArray, currRow, descriptions);
|
||||
|
||||
++currRow;
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getInverteds(String dimension, String value)
|
||||
{
|
||||
return new ConciseCompressedIndexedInts(index.getInvertedIndex(dimension, value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getMetricType(String metric)
|
||||
{
|
||||
MetricHolder holder = index.getMetricHolder(metric);
|
||||
if (holder == null) {
|
||||
return null;
|
||||
}
|
||||
return holder.getTypeName();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ProgressIndicator
|
||||
{
|
||||
public void progress();
|
||||
|
||||
public void start();
|
||||
|
||||
public void stop();
|
||||
|
||||
public void startSection(String section);
|
||||
|
||||
public void progressSection(String section, String message);
|
||||
|
||||
public void stopSection(String section);
|
||||
}
|
|
@ -27,6 +27,7 @@ import com.metamx.common.guava.CloseQuietly;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ComplexColumn;
|
||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||
import io.druid.segment.column.GenericColumn;
|
||||
|
@ -94,18 +95,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableDimensions()
|
||||
public Indexed<String> getDimensionNames()
|
||||
{
|
||||
return new ListIndexed<String>(availableDimensions, String.class);
|
||||
return new ListIndexed<>(availableDimensions, String.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableMetrics()
|
||||
public Indexed<String> getMetricNames()
|
||||
{
|
||||
final Set<String> columns = Sets.newLinkedHashSet(input.getColumnNames());
|
||||
final HashSet<String> dimensions = Sets.newHashSet(getAvailableDimensions());
|
||||
final HashSet<String> dimensions = Sets.newHashSet(getDimensionNames());
|
||||
|
||||
return new ListIndexed<String>(
|
||||
return new ListIndexed<>(
|
||||
Lists.newArrayList(Sets.difference(columns, dimensions)),
|
||||
String.class
|
||||
);
|
||||
|
@ -174,18 +175,18 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
final Object[] metrics;
|
||||
final Map<String, DictionaryEncodedColumn> dimensions;
|
||||
|
||||
final int numMetrics = getAvailableMetrics().size();
|
||||
final int numMetrics = getMetricNames().size();
|
||||
|
||||
int currRow = 0;
|
||||
boolean done = false;
|
||||
|
||||
{
|
||||
dimensions = Maps.newLinkedHashMap();
|
||||
for (String dim : getAvailableDimensions()) {
|
||||
for (String dim : getDimensionNames()) {
|
||||
dimensions.put(dim, input.getColumn(dim).getDictionaryEncoding());
|
||||
}
|
||||
|
||||
final Indexed<String> availableMetrics = getAvailableMetrics();
|
||||
final Indexed<String> availableMetrics = getMetricNames();
|
||||
metrics = new Object[availableMetrics.size()];
|
||||
for (int i = 0; i < metrics.length; ++i) {
|
||||
final Column column = input.getColumn(availableMetrics.get(i));
|
||||
|
@ -254,14 +255,8 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
}
|
||||
}
|
||||
|
||||
Map<String, String> descriptions = Maps.newHashMap();
|
||||
for (String columnName : input.getColumnNames()) {
|
||||
if (input.getColumn(columnName).getSpatialIndex() != null) {
|
||||
descriptions.put(columnName, "spatial");
|
||||
}
|
||||
}
|
||||
final Rowboat retVal = new Rowboat(
|
||||
timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow, descriptions
|
||||
timestamps.getLongSingleValueRow(currRow), dims, metricArray, currRow
|
||||
);
|
||||
|
||||
++currRow;
|
||||
|
@ -311,4 +306,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
throw new ISE("Unknown type[%s]", type);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return input.getColumn(column).getCapabilities();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,21 +37,17 @@ public class Rowboat implements Comparable<Rowboat>
|
|||
private final int rowNum;
|
||||
private final Map<Integer, TreeSet<Integer>> comprisedRows;
|
||||
|
||||
private Map<String, String> columnDescriptor;
|
||||
|
||||
public Rowboat(
|
||||
long timestamp,
|
||||
int[][] dims,
|
||||
Object[] metrics,
|
||||
int rowNum,
|
||||
Map<String, String> columnDescriptor
|
||||
int rowNum
|
||||
)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.dims = dims;
|
||||
this.metrics = metrics;
|
||||
this.rowNum = rowNum;
|
||||
this.columnDescriptor = columnDescriptor;
|
||||
|
||||
this.comprisedRows = Maps.newHashMap();
|
||||
}
|
||||
|
@ -91,11 +87,6 @@ public class Rowboat implements Comparable<Rowboat>
|
|||
return rowNum;
|
||||
}
|
||||
|
||||
public Map<String, String> getDescriptions()
|
||||
{
|
||||
return columnDescriptor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Rowboat rhs)
|
||||
{
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.segment;
|
|||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -51,15 +52,15 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableDimensions()
|
||||
public Indexed<String> getDimensionNames()
|
||||
{
|
||||
return baseAdapter.getAvailableDimensions();
|
||||
return baseAdapter.getDimensionNames();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableMetrics()
|
||||
public Indexed<String> getMetricNames()
|
||||
{
|
||||
return baseAdapter.getAvailableMetrics();
|
||||
return baseAdapter.getMetricNames();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -85,4 +86,10 @@ public class RowboatFilteringIndexAdapter implements IndexableAdapter
|
|||
{
|
||||
return baseAdapter.getMetricType(metric);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return baseAdapter.getCapabilities(column);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,4 +30,6 @@ public interface ColumnCapabilities
|
|||
public boolean hasBitmapIndexes();
|
||||
public boolean hasSpatialIndexes();
|
||||
public boolean hasMultipleValues();
|
||||
|
||||
public ColumnCapabilitiesImpl merge(ColumnCapabilities other);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.segment.column;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.common.ISE;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -109,4 +110,28 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilitiesImpl merge(ColumnCapabilities other)
|
||||
{
|
||||
if (other == null) {
|
||||
return this;
|
||||
}
|
||||
|
||||
if (type == null) {
|
||||
type = other.getType();
|
||||
}
|
||||
|
||||
if (!type.equals(other.getType())) {
|
||||
throw new ISE("Cannot merge columns of type[%s] and [%s]", type, other.getType());
|
||||
}
|
||||
|
||||
this.dictionaryEncoded |= other.isDictionaryEncoded();
|
||||
this.runLengthEncoded |= other.isRunLengthEncoded();
|
||||
this.hasInvertedIndexes |= other.hasBitmapIndexes();
|
||||
this.hasSpatialIndexes |= other.hasSpatialIndexes();
|
||||
this.hasMultipleValues |= other.hasMultipleValues();
|
||||
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,8 +38,6 @@ import java.util.Arrays;
|
|||
*/
|
||||
public class GenericIndexedWriter<T> implements Closeable
|
||||
{
|
||||
private static final byte[] EMPTY_ARRAY = new byte[]{};
|
||||
|
||||
private final IOPeon ioPeon;
|
||||
private final String filenameBase;
|
||||
private final ObjectStrategy<T> strategy;
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.IAE;
|
||||
|
||||
|
@ -44,6 +45,11 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
|||
return fromList(Ints.asList(array), maxValue);
|
||||
}
|
||||
|
||||
public static VSizeIndexedInts empty()
|
||||
{
|
||||
return fromList(Lists.<Integer>newArrayList(), 0);
|
||||
}
|
||||
|
||||
public static VSizeIndexedInts fromList(List<Integer> list, int maxValue)
|
||||
{
|
||||
int numBytes = getNumBytesForMax(maxValue);
|
||||
|
@ -57,7 +63,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
|||
if (val > maxValue) {
|
||||
throw new IAE("val[%d] > maxValue[%d], please don't lie about maxValue. i[%d]", val, maxValue, i);
|
||||
}
|
||||
|
||||
|
||||
byte[] intAsBytes = Ints.toByteArray(val);
|
||||
buffer.put(intAsBytes, intAsBytes.length - numBytes, numBytes);
|
||||
++i;
|
||||
|
@ -138,8 +144,8 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
|||
if (retVal == 0) {
|
||||
retVal = buffer.compareTo(o.buffer);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public int getNumBytes()
|
||||
|
@ -149,6 +155,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable<VSizeIndexedInt
|
|||
|
||||
public int getSerializedSize()
|
||||
{
|
||||
// version, numBytes, size, remaining
|
||||
return 1 + 1 + 4 + buffer.remaining();
|
||||
}
|
||||
|
||||
|
|
|
@ -142,4 +142,4 @@ public class VSizeIndexedWriter implements Closeable
|
|||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.segment.incremental;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.BiMap;
|
||||
import com.google.common.collect.HashBiMap;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -32,20 +32,24 @@ import com.google.common.primitives.Ints;
|
|||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.FloatColumnSelector;
|
||||
import io.druid.segment.ObjectColumnSelector;
|
||||
import io.druid.segment.TimestampColumnSelector;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||
import io.druid.segment.serde.ComplexMetricSerde;
|
||||
|
@ -54,13 +58,16 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ConcurrentNavigableMap;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
@ -68,36 +75,199 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class IncrementalIndex implements Iterable<Row>
|
||||
public class IncrementalIndex implements Iterable<Row>, Closeable
|
||||
{
|
||||
private static final Logger log = new Logger(IncrementalIndex.class);
|
||||
private static final Joiner JOINER = Joiner.on(",");
|
||||
private final long minTimestamp;
|
||||
private final QueryGranularity gran;
|
||||
private final List<Function<InputRow, InputRow>> rowTransformers;
|
||||
private final AggregatorFactory[] metrics;
|
||||
private final Map<String, Integer> metricIndexes;
|
||||
private final Map<String, String> metricTypes;
|
||||
private final ImmutableList<String> metricNames;
|
||||
private final BufferAggregator[] aggs;
|
||||
private final int[] aggPositionOffsets;
|
||||
private final int totalAggSize;
|
||||
private final LinkedHashMap<String, Integer> dimensionOrder;
|
||||
private final CopyOnWriteArrayList<String> dimensions;
|
||||
private final List<SpatialDimensionSchema> spatialDimensions;
|
||||
private final SpatialDimensionRowFormatter spatialDimensionRowFormatter;
|
||||
protected final CopyOnWriteArrayList<String> dimensions;
|
||||
private final DimensionHolder dimValues;
|
||||
private final ConcurrentSkipListMap<TimeAndDims, Aggregator[]> facts;
|
||||
private final Map<String, ColumnCapabilitiesImpl> columnCapabilities;
|
||||
private final ConcurrentNavigableMap<TimeAndDims, Integer> facts;
|
||||
private final ResourceHolder<ByteBuffer> bufferHolder;
|
||||
private volatile AtomicInteger numEntries = new AtomicInteger();
|
||||
// This is modified on add() in a critical section.
|
||||
private InputRow in;
|
||||
private ThreadLocal<InputRow> in = new ThreadLocal<>();
|
||||
|
||||
public IncrementalIndex(IncrementalIndexSchema incrementalIndexSchema)
|
||||
/**
|
||||
* Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that
|
||||
* should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics.
|
||||
*
|
||||
* @param incrementalIndexSchema
|
||||
* @param bufferPool
|
||||
* @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input
|
||||
* value for aggregators that return metrics other than float.
|
||||
*/
|
||||
public IncrementalIndex(
|
||||
IncrementalIndexSchema incrementalIndexSchema,
|
||||
StupidPool<ByteBuffer> bufferPool,
|
||||
final boolean deserializeComplexMetrics
|
||||
)
|
||||
{
|
||||
this.minTimestamp = incrementalIndexSchema.getMinTimestamp();
|
||||
this.gran = incrementalIndexSchema.getGran();
|
||||
this.metrics = incrementalIndexSchema.getMetrics();
|
||||
this.rowTransformers = Lists.newCopyOnWriteArrayList();
|
||||
|
||||
final ImmutableList.Builder<String> metricNamesBuilder = ImmutableList.builder();
|
||||
final ImmutableMap.Builder<String, Integer> metricIndexesBuilder = ImmutableMap.builder();
|
||||
final ImmutableMap.Builder<String, String> metricTypesBuilder = ImmutableMap.builder();
|
||||
this.aggs = new BufferAggregator[metrics.length];
|
||||
this.aggPositionOffsets = new int[metrics.length];
|
||||
int currAggSize = 0;
|
||||
for (int i = 0; i < metrics.length; i++) {
|
||||
final AggregatorFactory agg = metrics[i];
|
||||
aggs[i] = agg.factorizeBuffered(
|
||||
new ColumnSelectorFactory()
|
||||
{
|
||||
@Override
|
||||
public TimestampColumnSelector makeTimestampColumnSelector()
|
||||
{
|
||||
return new TimestampColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public long getTimestamp()
|
||||
{
|
||||
return in.get().getTimestampFromEpoch();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
return new FloatColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public float get()
|
||||
{
|
||||
return in.get().getFloatMetric(metricName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||
{
|
||||
final String typeName = agg.getTypeName();
|
||||
final String columnName = column.toLowerCase();
|
||||
|
||||
final ObjectColumnSelector<Object> rawColumnSelector = new ObjectColumnSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return in.get().getRaw(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
if (!deserializeComplexMetrics) {
|
||||
return rawColumnSelector;
|
||||
} else {
|
||||
if (typeName.equals("float")) {
|
||||
return rawColumnSelector;
|
||||
}
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
if (serde == null) {
|
||||
throw new ISE("Don't know how to handle type[%s]", typeName);
|
||||
}
|
||||
|
||||
final ComplexMetricExtractor extractor = serde.getExtractor();
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return extractor.extractedClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return extractor.extractValue(in.get(), columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
final List<String> dimensionValues = in.get().getDimension(dimensionName);
|
||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
||||
if (dimensionValues != null) {
|
||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
||||
vals.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new IndexedInts()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return vals.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return vals.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return vals.iterator();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
throw new UnsupportedOperationException("value cardinality is unknown in incremental index");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return in.get().getDimension(dimensionName).get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return in.get().getDimension(dimensionName).indexOf(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
);
|
||||
aggPositionOffsets[i] = currAggSize;
|
||||
currAggSize += agg.getMaxIntermediateSize();
|
||||
final String metricName = metrics[i].getName().toLowerCase();
|
||||
metricNamesBuilder.add(metricName);
|
||||
metricIndexesBuilder.put(metricName, i);
|
||||
|
@ -107,60 +277,118 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
metricIndexes = metricIndexesBuilder.build();
|
||||
metricTypes = metricTypesBuilder.build();
|
||||
|
||||
this.dimensionOrder = Maps.newLinkedHashMap();
|
||||
this.dimensions = new CopyOnWriteArrayList<String>();
|
||||
int index = 0;
|
||||
for (String dim : incrementalIndexSchema.getDimensions()) {
|
||||
dimensionOrder.put(dim, index++);
|
||||
dimensions.add(dim);
|
||||
}
|
||||
this.spatialDimensions = incrementalIndexSchema.getSpatialDimensions();
|
||||
this.spatialDimensionRowFormatter = new SpatialDimensionRowFormatter(spatialDimensions);
|
||||
this.totalAggSize = currAggSize;
|
||||
|
||||
this.dimensionOrder = Maps.newLinkedHashMap();
|
||||
this.dimensions = new CopyOnWriteArrayList<>();
|
||||
// This should really be more generic
|
||||
List<SpatialDimensionSchema> spatialDimensions = incrementalIndexSchema.getDimensionsSpec().getSpatialDimensions();
|
||||
if (!spatialDimensions.isEmpty()) {
|
||||
this.rowTransformers.add(new SpatialDimensionRowTransformer(spatialDimensions));
|
||||
}
|
||||
|
||||
this.columnCapabilities = Maps.newHashMap();
|
||||
for (Map.Entry<String, String> entry : metricTypes.entrySet()) {
|
||||
ValueType type;
|
||||
if (entry.getValue().equalsIgnoreCase("float")) {
|
||||
type = ValueType.FLOAT;
|
||||
} else {
|
||||
type = ValueType.COMPLEX;
|
||||
}
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setType(type);
|
||||
columnCapabilities.put(entry.getKey(), capabilities);
|
||||
}
|
||||
for (String dimension : dimensions) {
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setType(ValueType.STRING);
|
||||
columnCapabilities.put(dimension, capabilities);
|
||||
}
|
||||
for (SpatialDimensionSchema spatialDimension : spatialDimensions) {
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setType(ValueType.STRING);
|
||||
capabilities.setHasSpatialIndexes(true);
|
||||
columnCapabilities.put(spatialDimension.getDimName(), capabilities);
|
||||
}
|
||||
this.bufferHolder = bufferPool.take();
|
||||
this.dimValues = new DimensionHolder();
|
||||
this.facts = new ConcurrentSkipListMap<TimeAndDims, Aggregator[]>();
|
||||
this.facts = createFactsTable();
|
||||
}
|
||||
|
||||
protected ConcurrentNavigableMap<TimeAndDims, Integer> createFactsTable() {
|
||||
return new ConcurrentSkipListMap<>();
|
||||
}
|
||||
|
||||
public IncrementalIndex(
|
||||
long minTimestamp,
|
||||
QueryGranularity gran,
|
||||
final AggregatorFactory[] metrics
|
||||
final AggregatorFactory[] metrics,
|
||||
StupidPool<ByteBuffer> bufferPool
|
||||
)
|
||||
{
|
||||
this(
|
||||
new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp)
|
||||
.withQueryGranularity(gran)
|
||||
.withMetrics(metrics)
|
||||
.build()
|
||||
.build(),
|
||||
bufferPool,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
public int add(InputRow row)
|
||||
public IncrementalIndex(
|
||||
IncrementalIndexSchema incrementalIndexSchema,
|
||||
StupidPool<ByteBuffer> bufferPool
|
||||
)
|
||||
{
|
||||
// this is an ugly workaround to call ComplexMetricExtractor.extractValue at ingestion time
|
||||
return add(row, true);
|
||||
this(incrementalIndexSchema, bufferPool, true);
|
||||
}
|
||||
|
||||
public IncrementalIndex(
|
||||
long minTimestamp,
|
||||
QueryGranularity gran,
|
||||
final AggregatorFactory[] metrics,
|
||||
StupidPool<ByteBuffer> bufferPool,
|
||||
boolean deserializeComplexMetrics
|
||||
)
|
||||
{
|
||||
this(
|
||||
new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp)
|
||||
.withQueryGranularity(gran)
|
||||
.withMetrics(metrics)
|
||||
.build(),
|
||||
bufferPool,
|
||||
deserializeComplexMetrics
|
||||
);
|
||||
}
|
||||
|
||||
public InputRow formatRow(InputRow row)
|
||||
{
|
||||
for (Function<InputRow, InputRow> rowTransformer : rowTransformers) {
|
||||
row = rowTransformer.apply(row);
|
||||
}
|
||||
|
||||
if (row == null) {
|
||||
throw new IAE("Row is null? How can this be?!");
|
||||
}
|
||||
return row;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
||||
* update that row instead of inserting a new one.
|
||||
*
|
||||
*
|
||||
* <p/>
|
||||
* <p/>
|
||||
* Calls to add() are thread safe.
|
||||
*
|
||||
* Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that
|
||||
* should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics.
|
||||
* <p/>
|
||||
*
|
||||
* @param row the row of data to add
|
||||
* @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input
|
||||
* value for aggregators that return metrics other than float.
|
||||
*
|
||||
* @return the number of rows in the data set after adding the InputRow
|
||||
*/
|
||||
public int add(InputRow row, final boolean deserializeComplexMetrics)
|
||||
public int add(InputRow row)
|
||||
{
|
||||
row = spatialDimensionRowFormatter.formatRow(row);
|
||||
|
||||
row = formatRow(row);
|
||||
if (row.getTimestampFromEpoch() < minTimestamp) {
|
||||
throw new IAE("Cannot add row[%s] because it is below the minTimestamp[%s]", row, new DateTime(minTimestamp));
|
||||
}
|
||||
|
@ -174,6 +402,18 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
for (String dimension : rowDimensions) {
|
||||
dimension = dimension.toLowerCase();
|
||||
List<String> dimensionValues = row.getDimension(dimension);
|
||||
|
||||
// Set column capabilities as data is coming in
|
||||
ColumnCapabilitiesImpl capabilities = columnCapabilities.get(dimension);
|
||||
if (capabilities == null) {
|
||||
capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setType(ValueType.STRING);
|
||||
columnCapabilities.put(dimension, capabilities);
|
||||
}
|
||||
if (dimensionValues.size() > 1) {
|
||||
capabilities.setHasMultipleValues(true);
|
||||
}
|
||||
|
||||
Integer index = dimensionOrder.get(dimension);
|
||||
if (index == null) {
|
||||
dimensionOrder.put(dimension, dimensionOrder.size());
|
||||
|
@ -201,173 +441,30 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
}
|
||||
|
||||
final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims);
|
||||
|
||||
Aggregator[] aggs = facts.get(key);
|
||||
if (aggs == null) {
|
||||
aggs = new Aggregator[metrics.length];
|
||||
|
||||
for (int i = 0; i < metrics.length; ++i) {
|
||||
final AggregatorFactory agg = metrics[i];
|
||||
aggs[i] =
|
||||
agg.factorize(
|
||||
new ColumnSelectorFactory()
|
||||
{
|
||||
@Override
|
||||
public TimestampColumnSelector makeTimestampColumnSelector()
|
||||
{
|
||||
return new TimestampColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public long getTimestamp()
|
||||
{
|
||||
return in.getTimestampFromEpoch();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public FloatColumnSelector makeFloatColumnSelector(String columnName)
|
||||
{
|
||||
final String metricName = columnName.toLowerCase();
|
||||
return new FloatColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public float get()
|
||||
{
|
||||
return in.getFloatMetric(metricName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectColumnSelector makeObjectColumnSelector(String column)
|
||||
{
|
||||
final String typeName = agg.getTypeName();
|
||||
final String columnName = column.toLowerCase();
|
||||
|
||||
final ObjectColumnSelector<Object> rawColumnSelector = new ObjectColumnSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return in.getRaw(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
if(!deserializeComplexMetrics) {
|
||||
return rawColumnSelector;
|
||||
} else {
|
||||
if (typeName.equals("float")) {
|
||||
return rawColumnSelector;
|
||||
}
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
if (serde == null) {
|
||||
throw new ISE("Don't know how to handle type[%s]", typeName);
|
||||
}
|
||||
|
||||
final ComplexMetricExtractor extractor = serde.getExtractor();
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return extractor.extractedClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return extractor.extractValue(in, columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
final List<String> dimensionValues = in.getDimension(dimensionName);
|
||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
||||
if (dimensionValues != null) {
|
||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
||||
vals.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new IndexedInts()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return vals.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return vals.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return vals.iterator();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
throw new UnsupportedOperationException("value cardinality is unknown in incremental index");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return in.getDimension(dimensionName).get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return in.getDimension(dimensionName).indexOf(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
);
|
||||
}
|
||||
|
||||
Aggregator[] prev = facts.putIfAbsent(key, aggs);
|
||||
if (prev != null) {
|
||||
aggs = prev;
|
||||
} else {
|
||||
numEntries.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
Integer rowOffset;
|
||||
synchronized (this) {
|
||||
in = row;
|
||||
for (Aggregator agg : aggs) {
|
||||
agg.aggregate();
|
||||
rowOffset = totalAggSize * numEntries.get();
|
||||
final Integer prev = facts.putIfAbsent(key, rowOffset);
|
||||
if (prev != null) {
|
||||
rowOffset = prev;
|
||||
} else {
|
||||
if (rowOffset + totalAggSize > bufferHolder.get().limit()) {
|
||||
facts.remove(key);
|
||||
throw new ISE("Buffer full, cannot add more rows! Current rowSize[%,d].", numEntries.get());
|
||||
}
|
||||
numEntries.incrementAndGet();
|
||||
for (int i = 0; i < aggs.length; i++) {
|
||||
aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i));
|
||||
}
|
||||
}
|
||||
in = null;
|
||||
}
|
||||
in.set(row);
|
||||
for (int i = 0; i < aggs.length; i++) {
|
||||
synchronized (aggs[i]) {
|
||||
aggs[i].aggregate(bufferHolder.get(), getMetricPosition(rowOffset, i));
|
||||
}
|
||||
}
|
||||
in.set(null);
|
||||
return numEntries.get();
|
||||
}
|
||||
|
||||
|
@ -398,11 +495,9 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
int count = 0;
|
||||
for (String dimValue : dimValues) {
|
||||
String canonicalDimValue = dimLookup.get(dimValue);
|
||||
if (canonicalDimValue == null) {
|
||||
canonicalDimValue = dimValue;
|
||||
if (!dimLookup.contains(canonicalDimValue)) {
|
||||
dimLookup.add(dimValue);
|
||||
}
|
||||
|
||||
retVal[count] = canonicalDimValue;
|
||||
count++;
|
||||
}
|
||||
|
@ -421,16 +516,6 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
return dimensions;
|
||||
}
|
||||
|
||||
public List<SpatialDimensionSchema> getSpatialDimensions()
|
||||
{
|
||||
return spatialDimensions;
|
||||
}
|
||||
|
||||
public SpatialDimensionRowFormatter getSpatialDimensionRowFormatter()
|
||||
{
|
||||
return spatialDimensionRowFormatter;
|
||||
}
|
||||
|
||||
public String getMetricType(String metric)
|
||||
{
|
||||
return metricTypes.get(metric);
|
||||
|
@ -481,12 +566,32 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
return metricIndexes.get(metricName);
|
||||
}
|
||||
|
||||
ConcurrentSkipListMap<TimeAndDims, Aggregator[]> getFacts()
|
||||
int getMetricPosition(int rowOffset, int metricIndex)
|
||||
{
|
||||
return rowOffset + aggPositionOffsets[metricIndex];
|
||||
}
|
||||
|
||||
ByteBuffer getMetricBuffer()
|
||||
{
|
||||
return bufferHolder.get();
|
||||
}
|
||||
|
||||
BufferAggregator getAggregator(int metricIndex)
|
||||
{
|
||||
return aggs[metricIndex];
|
||||
}
|
||||
|
||||
ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return columnCapabilities.get(column);
|
||||
}
|
||||
|
||||
ConcurrentNavigableMap<TimeAndDims, Integer> getFacts()
|
||||
{
|
||||
return facts;
|
||||
}
|
||||
|
||||
ConcurrentNavigableMap<TimeAndDims, Aggregator[]> getSubMap(TimeAndDims start, TimeAndDims end)
|
||||
ConcurrentNavigableMap<TimeAndDims, Integer> getSubMap(TimeAndDims start, TimeAndDims end)
|
||||
{
|
||||
return facts.subMap(start, end);
|
||||
}
|
||||
|
@ -506,13 +611,13 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
{
|
||||
return Iterators.transform(
|
||||
facts.entrySet().iterator(),
|
||||
new Function<Map.Entry<TimeAndDims, Aggregator[]>, Row>()
|
||||
new Function<Map.Entry<TimeAndDims, Integer>, Row>()
|
||||
{
|
||||
@Override
|
||||
public Row apply(final Map.Entry<TimeAndDims, Aggregator[]> input)
|
||||
public Row apply(final Map.Entry<TimeAndDims, Integer> input)
|
||||
{
|
||||
final TimeAndDims timeAndDims = input.getKey();
|
||||
final Aggregator[] aggregators = input.getValue();
|
||||
final int rowOffset = input.getValue();
|
||||
|
||||
String[][] theDims = timeAndDims.getDims();
|
||||
|
||||
|
@ -524,8 +629,8 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < aggregators.length; ++i) {
|
||||
theVals.put(metrics[i].getName(), aggregators[i].get());
|
||||
for (int i = 0; i < aggs.length; ++i) {
|
||||
theVals.put(metrics[i].getName(), aggs[i].get(bufferHolder.get(), getMetricPosition(rowOffset, i)));
|
||||
}
|
||||
|
||||
if (postAggs != null) {
|
||||
|
@ -542,7 +647,18 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
};
|
||||
}
|
||||
|
||||
static class DimensionHolder
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
try {
|
||||
bufferHolder.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
class DimensionHolder
|
||||
{
|
||||
private final Map<String, DimDim> dimensions;
|
||||
|
||||
|
@ -560,7 +676,7 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
{
|
||||
DimDim holder = dimensions.get(dimension);
|
||||
if (holder == null) {
|
||||
holder = new DimDim();
|
||||
holder = createDimDim(dimension);
|
||||
dimensions.put(dimension, holder);
|
||||
} else {
|
||||
throw new ISE("dimension[%s] already existed even though add() was called!?", dimension);
|
||||
|
@ -574,6 +690,10 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
}
|
||||
}
|
||||
|
||||
protected DimDim createDimDim(String dimension){
|
||||
return new DimDimImpl();
|
||||
}
|
||||
|
||||
static class TimeAndDims implements Comparable<TimeAndDims>
|
||||
{
|
||||
private final long timestamp;
|
||||
|
@ -659,27 +779,51 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
}
|
||||
}
|
||||
|
||||
static class DimDim
|
||||
static interface DimDim
|
||||
{
|
||||
private final Map<String, String> poorMansInterning = Maps.newConcurrentMap();
|
||||
public String get(String value);
|
||||
|
||||
public int getId(String value);
|
||||
|
||||
public String getValue(int id);
|
||||
|
||||
public boolean contains(String value);
|
||||
|
||||
public int size();
|
||||
|
||||
public int add(String value);
|
||||
|
||||
public int getSortedId(String value);
|
||||
|
||||
public String getSortedValue(int index);
|
||||
|
||||
public void sort();
|
||||
|
||||
public boolean compareCannonicalValues(String s1, String s2);
|
||||
}
|
||||
|
||||
private static class DimDimImpl implements DimDim{
|
||||
private final Map<String, Integer> falseIds;
|
||||
private final Map<Integer, String> falseIdsReverse;
|
||||
private volatile String[] sortedVals = null;
|
||||
final ConcurrentMap<String, String> poorMansInterning = Maps.newConcurrentMap();
|
||||
|
||||
public DimDim()
|
||||
|
||||
public DimDimImpl()
|
||||
{
|
||||
BiMap<String, Integer> biMap = Maps.synchronizedBiMap(HashBiMap.<String, Integer>create());
|
||||
falseIds = biMap;
|
||||
falseIdsReverse = biMap.inverse();
|
||||
BiMap<String, Integer> biMap = Maps.synchronizedBiMap(HashBiMap.<String, Integer>create());
|
||||
falseIds = biMap;
|
||||
falseIdsReverse = biMap.inverse();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the interned String value to allow fast comparisons using `==` instead of `.equals()`
|
||||
* @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String)
|
||||
*/
|
||||
public String get(String value)
|
||||
public String get(String str)
|
||||
{
|
||||
return value == null ? null : poorMansInterning.get(value);
|
||||
String prev = poorMansInterning.putIfAbsent(str, str);
|
||||
return prev != null ? prev : str;
|
||||
}
|
||||
|
||||
public int getId(String value)
|
||||
|
@ -692,20 +836,21 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
return falseIdsReverse.get(id);
|
||||
}
|
||||
|
||||
public boolean contains(String value)
|
||||
{
|
||||
return falseIds.containsKey(value);
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
return poorMansInterning.size();
|
||||
return falseIds.size();
|
||||
}
|
||||
|
||||
public Set<String> keySet()
|
||||
public synchronized int add(String value)
|
||||
{
|
||||
return poorMansInterning.keySet();
|
||||
}
|
||||
|
||||
public synchronized void add(String value)
|
||||
{
|
||||
poorMansInterning.put(value, value);
|
||||
falseIds.put(value, falseIds.size());
|
||||
int id = falseIds.size();
|
||||
falseIds.put(value, id);
|
||||
return id;
|
||||
}
|
||||
|
||||
public int getSortedId(String value)
|
||||
|
@ -739,5 +884,10 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
throw new ISE("Call sort() before calling the getSorted* methods.");
|
||||
}
|
||||
}
|
||||
|
||||
public boolean compareCannonicalValues(String s1, String s2)
|
||||
{
|
||||
return s1 ==s2;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,10 +23,9 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.segment.IndexableAdapter;
|
||||
import io.druid.segment.Rowboat;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.data.EmptyIndexedInts;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
|
@ -45,10 +44,8 @@ import java.util.Map;
|
|||
public class IncrementalIndexAdapter implements IndexableAdapter
|
||||
{
|
||||
private static final Logger log = new Logger(IncrementalIndexAdapter.class);
|
||||
|
||||
private final Interval dataInterval;
|
||||
private final IncrementalIndex index;
|
||||
|
||||
private final Map<String, Map<String, ConciseSet>> invertedIndexes;
|
||||
|
||||
public IncrementalIndexAdapter(
|
||||
|
@ -114,13 +111,13 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableDimensions()
|
||||
public Indexed<String> getDimensionNames()
|
||||
{
|
||||
return new ListIndexed<String>(index.getDimensions(), String.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableMetrics()
|
||||
public Indexed<String> getMetricNames()
|
||||
{
|
||||
return new ListIndexed<String>(index.getMetricNames(), String.class);
|
||||
}
|
||||
|
@ -171,18 +168,18 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
return FunctionalIterable
|
||||
.create(index.getFacts().entrySet())
|
||||
.transform(
|
||||
new Function<Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]>, Rowboat>()
|
||||
new Function<Map.Entry<IncrementalIndex.TimeAndDims, Integer>, Rowboat>()
|
||||
{
|
||||
int count = 0;
|
||||
|
||||
@Override
|
||||
public Rowboat apply(
|
||||
@Nullable Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]> input
|
||||
@Nullable Map.Entry<IncrementalIndex.TimeAndDims, Integer> input
|
||||
)
|
||||
{
|
||||
final IncrementalIndex.TimeAndDims timeAndDims = input.getKey();
|
||||
final String[][] dimValues = timeAndDims.getDims();
|
||||
final Aggregator[] aggs = input.getValue();
|
||||
final int rowOffset = input.getValue();
|
||||
|
||||
int[][] dims = new int[dimValues.length][];
|
||||
for (String dimension : index.getDimensions()) {
|
||||
|
@ -205,21 +202,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
}
|
||||
}
|
||||
|
||||
Object[] metrics = new Object[aggs.length];
|
||||
for (int i = 0; i < aggs.length; i++) {
|
||||
metrics[i] = aggs[i].get();
|
||||
Object[] metrics = new Object[index.getMetricAggs().length];
|
||||
for (int i = 0; i < metrics.length; i++) {
|
||||
metrics[i] = index.getAggregator(i)
|
||||
.get(index.getMetricBuffer(), index.getMetricPosition(rowOffset, i));
|
||||
}
|
||||
|
||||
Map<String, String> description = Maps.newHashMap();
|
||||
for (SpatialDimensionSchema spatialDimensionSchema : index.getSpatialDimensions()) {
|
||||
description.put(spatialDimensionSchema.getDimName(), "spatial");
|
||||
}
|
||||
return new Rowboat(
|
||||
timeAndDims.getTimestamp(),
|
||||
dims,
|
||||
metrics,
|
||||
count++,
|
||||
description
|
||||
count++
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -289,4 +282,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
{
|
||||
return index.getMetricType(metric);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return index.getCapabilities(column);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,39 +19,30 @@
|
|||
|
||||
package io.druid.segment.incremental;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IncrementalIndexSchema
|
||||
{
|
||||
private final long minTimestamp;
|
||||
private final QueryGranularity gran;
|
||||
private final List<String> dimensions;
|
||||
private final List<SpatialDimensionSchema> spatialDimensions;
|
||||
private final DimensionsSpec dimensionsSpec;
|
||||
private final AggregatorFactory[] metrics;
|
||||
|
||||
public IncrementalIndexSchema(
|
||||
long minTimestamp,
|
||||
QueryGranularity gran,
|
||||
List<String> dimensions,
|
||||
List<SpatialDimensionSchema> spatialDimensions,
|
||||
DimensionsSpec dimensionsSpec,
|
||||
AggregatorFactory[] metrics
|
||||
)
|
||||
{
|
||||
this.minTimestamp = minTimestamp;
|
||||
this.gran = gran;
|
||||
this.dimensions = dimensions;
|
||||
this.spatialDimensions = spatialDimensions;
|
||||
this.dimensionsSpec = dimensionsSpec;
|
||||
this.metrics = metrics;
|
||||
}
|
||||
|
||||
|
@ -65,14 +56,9 @@ public class IncrementalIndexSchema
|
|||
return gran;
|
||||
}
|
||||
|
||||
public List<String> getDimensions()
|
||||
public DimensionsSpec getDimensionsSpec()
|
||||
{
|
||||
return dimensions;
|
||||
}
|
||||
|
||||
public List<SpatialDimensionSchema> getSpatialDimensions()
|
||||
{
|
||||
return spatialDimensions;
|
||||
return dimensionsSpec;
|
||||
}
|
||||
|
||||
public AggregatorFactory[] getMetrics()
|
||||
|
@ -84,16 +70,14 @@ public class IncrementalIndexSchema
|
|||
{
|
||||
private long minTimestamp;
|
||||
private QueryGranularity gran;
|
||||
private List<String> dimensions;
|
||||
private List<SpatialDimensionSchema> spatialDimensions;
|
||||
private DimensionsSpec dimensionsSpec;
|
||||
private AggregatorFactory[] metrics;
|
||||
|
||||
public Builder()
|
||||
{
|
||||
this.minTimestamp = 0L;
|
||||
this.gran = QueryGranularity.NONE;
|
||||
this.dimensions = Lists.newArrayList();
|
||||
this.spatialDimensions = Lists.newArrayList();
|
||||
this.dimensionsSpec = new DimensionsSpec(null, null, null);
|
||||
this.metrics = new AggregatorFactory[]{};
|
||||
}
|
||||
|
||||
|
@ -109,44 +93,25 @@ public class IncrementalIndexSchema
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder withDimensions(Iterable<String> dimensions)
|
||||
public Builder withDimensionsSpec(DimensionsSpec dimensionsSpec)
|
||||
{
|
||||
this.dimensions = Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
dimensions, new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String input)
|
||||
{
|
||||
return input.toLowerCase();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
Collections.sort(this.dimensions);
|
||||
this.dimensionsSpec = dimensionsSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSpatialDimensions(InputRowParser parser)
|
||||
public Builder withDimensionsSpec(InputRowParser parser)
|
||||
{
|
||||
if (parser != null
|
||||
&& parser.getParseSpec() != null
|
||||
&& parser.getParseSpec().getDimensionsSpec() != null
|
||||
&& parser.getParseSpec().getDimensionsSpec().getSpatialDimensions() != null) {
|
||||
this.spatialDimensions = parser.getParseSpec().getDimensionsSpec().getSpatialDimensions();
|
||||
&& parser.getParseSpec().getDimensionsSpec() != null) {
|
||||
this.dimensionsSpec = parser.getParseSpec().getDimensionsSpec();
|
||||
} else {
|
||||
this.spatialDimensions = Lists.newArrayList();
|
||||
this.dimensionsSpec = new DimensionsSpec(null, null, null);
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSpatialDimensions(List<SpatialDimensionSchema> spatialDimensions)
|
||||
{
|
||||
this.spatialDimensions = spatialDimensions;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMetrics(AggregatorFactory[] metrics)
|
||||
{
|
||||
this.metrics = metrics;
|
||||
|
@ -156,7 +121,7 @@ public class IncrementalIndexSchema
|
|||
public IncrementalIndexSchema build()
|
||||
{
|
||||
return new IncrementalIndexSchema(
|
||||
minTimestamp, gran, dimensions, spatialDimensions, metrics
|
||||
minTimestamp, gran, dimensionsSpec, metrics
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import com.metamx.common.guava.Sequence;
|
|||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.QueryInterruptedException;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
|
@ -62,7 +62,6 @@ import java.util.concurrent.ConcurrentNavigableMap;
|
|||
public class IncrementalIndexStorageAdapter implements StorageAdapter
|
||||
{
|
||||
private static final Splitter SPLITTER = Splitter.on(",");
|
||||
|
||||
private final IncrementalIndex index;
|
||||
|
||||
public IncrementalIndexStorageAdapter(
|
||||
|
@ -169,8 +168,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
|
||||
return new Cursor()
|
||||
{
|
||||
private Iterator<Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]>> baseIter;
|
||||
private ConcurrentNavigableMap<IncrementalIndex.TimeAndDims, Aggregator[]> cursorMap;
|
||||
private Iterator<Map.Entry<IncrementalIndex.TimeAndDims, Integer>> baseIter;
|
||||
private ConcurrentNavigableMap<IncrementalIndex.TimeAndDims, Integer> cursorMap;
|
||||
final DateTime time;
|
||||
int numAdvanced = -1;
|
||||
boolean done;
|
||||
|
@ -367,13 +366,17 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
final int metricIndex = metricIndexInt;
|
||||
final BufferAggregator agg = index.getAggregator(metricIndex);
|
||||
|
||||
return new FloatColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public float get()
|
||||
{
|
||||
return currEntry.getValue()[metricIndex].getFloat();
|
||||
return agg.getFloat(
|
||||
index.getMetricBuffer(),
|
||||
index.getMetricPosition(currEntry.getValue(), metricIndex)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -388,7 +391,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
final int metricIndex = metricIndexInt;
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(index.getMetricType(columnName));
|
||||
|
||||
final BufferAggregator agg = index.getAggregator(metricIndex);
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
@Override
|
||||
|
@ -400,7 +403,10 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return currEntry.getValue()[metricIndex].get();
|
||||
return agg.get(
|
||||
index.getMetricBuffer(),
|
||||
index.getMetricPosition(currEntry.getValue(), metricIndex)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -427,11 +433,9 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
final String[] dimVals = dims[dimensionIndex];
|
||||
if (dimVals.length == 1) {
|
||||
return dimVals[0];
|
||||
}
|
||||
else if (dimVals.length == 0) {
|
||||
} else if (dimVals.length == 0) {
|
||||
return null;
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
return dimVals;
|
||||
}
|
||||
}
|
||||
|
@ -455,14 +459,14 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
|
||||
private static class EntryHolder
|
||||
{
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]> currEntry = null;
|
||||
Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry = null;
|
||||
|
||||
public Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]> get()
|
||||
public Map.Entry<IncrementalIndex.TimeAndDims, Integer> get()
|
||||
{
|
||||
return currEntry;
|
||||
}
|
||||
|
||||
public void set(Map.Entry<IncrementalIndex.TimeAndDims, Aggregator[]> currEntry)
|
||||
public void set(Map.Entry<IncrementalIndex.TimeAndDims, Integer> currEntry)
|
||||
{
|
||||
this.currEntry = currEntry;
|
||||
this.currEntry = currEntry;
|
||||
|
@ -473,7 +477,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
return currEntry.getKey();
|
||||
}
|
||||
|
||||
public Aggregator[] getValue()
|
||||
public Integer getValue()
|
||||
{
|
||||
return currEntry.getValue();
|
||||
}
|
||||
|
@ -497,8 +501,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
if (dimIndexObject == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
String idObject = index.getDimension(dimension.toLowerCase()).get(value);
|
||||
if (idObject == null) {
|
||||
final IncrementalIndex.DimDim dimDim = index.getDimension(dimension.toLowerCase());
|
||||
if (!dimDim.contains(value)) {
|
||||
if (value == null || "".equals(value)) {
|
||||
final int dimIndex = dimIndexObject;
|
||||
|
||||
|
@ -519,7 +523,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
final int dimIndex = dimIndexObject;
|
||||
final String id = idObject;
|
||||
final String id = dimDim.get(value);
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
|
@ -532,11 +536,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
for (String dimVal : dims[dimIndex]) {
|
||||
/**
|
||||
* using == here instead of .equals() to speed up lookups made possible by
|
||||
* {@link io.druid.segment.incremental.IncrementalIndex.DimDim#poorMansInterning}
|
||||
*/
|
||||
if (id == dimVal) {
|
||||
if (dimDim.compareCannonicalValues(id,dimVal)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,274 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.segment.incremental;
|
||||
|
||||
|
||||
import com.metamx.common.ISE;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import org.mapdb.BTreeKeySerializer;
|
||||
import org.mapdb.DB;
|
||||
import org.mapdb.DBMaker;
|
||||
import org.mapdb.Serializer;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.WeakHashMap;
|
||||
import java.util.concurrent.ConcurrentNavigableMap;
|
||||
|
||||
public class OffheapIncrementalIndex extends IncrementalIndex
|
||||
{
|
||||
private volatile DB db;
|
||||
private volatile DB factsDb;
|
||||
|
||||
public OffheapIncrementalIndex(
|
||||
IncrementalIndexSchema incrementalIndexSchema,
|
||||
StupidPool<ByteBuffer> bufferPool
|
||||
)
|
||||
{
|
||||
super(incrementalIndexSchema, bufferPool);
|
||||
}
|
||||
|
||||
public OffheapIncrementalIndex(
|
||||
long minTimestamp,
|
||||
QueryGranularity gran,
|
||||
final AggregatorFactory[] metrics,
|
||||
StupidPool<ByteBuffer> bufferPool,
|
||||
boolean deserializeComplexMetrics
|
||||
|
||||
)
|
||||
{
|
||||
super(minTimestamp, gran, metrics, bufferPool, deserializeComplexMetrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized ConcurrentNavigableMap<TimeAndDims, Integer> createFactsTable()
|
||||
{
|
||||
if (factsDb == null) {
|
||||
final DBMaker dbMaker = DBMaker.newMemoryDirectDB()
|
||||
.transactionDisable()
|
||||
.asyncWriteEnable()
|
||||
.cacheSoftRefEnable();
|
||||
factsDb = dbMaker.make();
|
||||
db = dbMaker.make();
|
||||
}
|
||||
final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this);
|
||||
return factsDb.createTreeMap("__facts" + UUID.randomUUID())
|
||||
.keySerializer(timeAndDimsSerializer)
|
||||
.comparator(timeAndDimsSerializer.getComparator())
|
||||
.valueSerializer(Serializer.INTEGER)
|
||||
.make();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DimDim createDimDim(String dimension)
|
||||
{
|
||||
return new OffheapDimDim(dimension);
|
||||
}
|
||||
|
||||
public static class TimeAndDimsSerializer extends BTreeKeySerializer<TimeAndDims> implements Serializable
|
||||
{
|
||||
private final TimeAndDimsComparator comparator;
|
||||
private final transient IncrementalIndex incrementalIndex;
|
||||
|
||||
TimeAndDimsSerializer(IncrementalIndex incrementalIndex)
|
||||
{
|
||||
this.comparator = new TimeAndDimsComparator();
|
||||
this.incrementalIndex = incrementalIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(DataOutput out, int start, int end, Object[] keys) throws IOException
|
||||
{
|
||||
for (int i = start; i < end; i++) {
|
||||
TimeAndDims timeAndDim = (TimeAndDims) keys[i];
|
||||
out.writeLong(timeAndDim.getTimestamp());
|
||||
out.writeInt(timeAndDim.getDims().length);
|
||||
int index = 0;
|
||||
for (String[] dims : timeAndDim.getDims()) {
|
||||
if (dims == null) {
|
||||
out.write(-1);
|
||||
} else {
|
||||
DimDim dimDim = incrementalIndex.getDimension(incrementalIndex.dimensions.get(index));
|
||||
out.writeInt(dims.length);
|
||||
for (String value : dims) {
|
||||
out.writeInt(dimDim.getId(value));
|
||||
}
|
||||
}
|
||||
index++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] deserialize(DataInput in, int start, int end, int size) throws IOException
|
||||
{
|
||||
Object[] ret = new Object[size];
|
||||
for (int i = start; i < end; i++) {
|
||||
final long timeStamp = in.readLong();
|
||||
final String[][] dims = new String[in.readInt()][];
|
||||
for (int k = 0; k < dims.length; k++) {
|
||||
int len = in.readInt();
|
||||
if (len != -1) {
|
||||
DimDim dimDim = incrementalIndex.getDimension(incrementalIndex.dimensions.get(k));
|
||||
String[] col = new String[len];
|
||||
for (int l = 0; l < col.length; l++) {
|
||||
col[l] = dimDim.get(dimDim.getValue(in.readInt()));
|
||||
}
|
||||
dims[k] = col;
|
||||
}
|
||||
}
|
||||
ret[i] = new TimeAndDims(timeStamp, dims);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<TimeAndDims> getComparator()
|
||||
{
|
||||
return comparator;
|
||||
}
|
||||
}
|
||||
|
||||
public static class TimeAndDimsComparator implements Comparator, Serializable
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o1, Object o2)
|
||||
{
|
||||
return ((TimeAndDims) o1).compareTo((TimeAndDims) o2);
|
||||
}
|
||||
}
|
||||
|
||||
private class OffheapDimDim implements DimDim
|
||||
{
|
||||
private final Map<String, Integer> falseIds;
|
||||
private final Map<Integer, String> falseIdsReverse;
|
||||
private final WeakHashMap<String, WeakReference<String>> cache =
|
||||
new WeakHashMap();
|
||||
private volatile String[] sortedVals = null;
|
||||
// size on MapDB is slow so maintain a count here
|
||||
private volatile int size = 0;
|
||||
|
||||
public OffheapDimDim(String dimension)
|
||||
{
|
||||
falseIds = db.createHashMap(dimension)
|
||||
.keySerializer(Serializer.STRING)
|
||||
.valueSerializer(Serializer.INTEGER)
|
||||
.make();
|
||||
falseIdsReverse = db.createHashMap(dimension + "_inverse")
|
||||
.keySerializer(Serializer.INTEGER)
|
||||
.valueSerializer(Serializer.STRING)
|
||||
.make();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the interned String value to allow fast comparisons using `==` instead of `.equals()`
|
||||
*
|
||||
* @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String)
|
||||
*/
|
||||
public String get(String str)
|
||||
{
|
||||
final WeakReference<String> cached = cache.get(str);
|
||||
if (cached != null) {
|
||||
final String value = cached.get();
|
||||
if (value != null) {
|
||||
return value;
|
||||
}
|
||||
}
|
||||
cache.put(str, new WeakReference(str));
|
||||
return str;
|
||||
}
|
||||
|
||||
public int getId(String value)
|
||||
{
|
||||
return falseIds.get(value);
|
||||
}
|
||||
|
||||
public String getValue(int id)
|
||||
{
|
||||
return falseIdsReverse.get(id);
|
||||
}
|
||||
|
||||
public boolean contains(String value)
|
||||
{
|
||||
return falseIds.containsKey(value);
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
public synchronized int add(String value)
|
||||
{
|
||||
int id = size++;
|
||||
falseIds.put(value, id);
|
||||
falseIdsReverse.put(id, value);
|
||||
return id;
|
||||
}
|
||||
|
||||
public int getSortedId(String value)
|
||||
{
|
||||
assertSorted();
|
||||
return Arrays.binarySearch(sortedVals, value);
|
||||
}
|
||||
|
||||
public String getSortedValue(int index)
|
||||
{
|
||||
assertSorted();
|
||||
return sortedVals[index];
|
||||
}
|
||||
|
||||
public void sort()
|
||||
{
|
||||
if (sortedVals == null) {
|
||||
sortedVals = new String[falseIds.size()];
|
||||
|
||||
int index = 0;
|
||||
for (String value : falseIds.keySet()) {
|
||||
sortedVals[index++] = value;
|
||||
}
|
||||
Arrays.sort(sortedVals);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertSorted()
|
||||
{
|
||||
if (sortedVals == null) {
|
||||
throw new ISE("Call sort() before calling the getSorted* methods.");
|
||||
}
|
||||
}
|
||||
|
||||
public boolean compareCannonicalValues(String s1, String s2)
|
||||
{
|
||||
return s1.equals(s2);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -44,31 +44,22 @@ import java.util.Set;
|
|||
/**
|
||||
* We throw away all invalid spatial dimensions
|
||||
*/
|
||||
public class SpatialDimensionRowFormatter
|
||||
public class SpatialDimensionRowTransformer implements Function<InputRow, InputRow>
|
||||
{
|
||||
private static final Joiner JOINER = Joiner.on(",");
|
||||
private static final Splitter SPLITTER = Splitter.on(",");
|
||||
|
||||
private final List<SpatialDimensionSchema> spatialDimensions;
|
||||
private final Set<String> spatialDimNames;
|
||||
private final Map<String, SpatialDimensionSchema> spatialDimensionMap;
|
||||
private final Set<String> spatialPartialDimNames;
|
||||
|
||||
public SpatialDimensionRowFormatter(List<SpatialDimensionSchema> spatialDimensions)
|
||||
public SpatialDimensionRowTransformer(List<SpatialDimensionSchema> spatialDimensions)
|
||||
{
|
||||
this.spatialDimensions = spatialDimensions;
|
||||
this.spatialDimNames = Sets.newHashSet(
|
||||
Lists.transform(
|
||||
spatialDimensions,
|
||||
new Function<SpatialDimensionSchema, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(SpatialDimensionSchema input)
|
||||
{
|
||||
return input.getDimName();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
this.spatialDimensionMap = Maps.newHashMap();
|
||||
for (SpatialDimensionSchema spatialDimension : spatialDimensions) {
|
||||
if (this.spatialDimensionMap.put(spatialDimension.getDimName(), spatialDimension) != null) {
|
||||
throw new ISE("Duplicate spatial dimension names found! Check your schema yo!");
|
||||
}
|
||||
}
|
||||
this.spatialPartialDimNames = Sets.newHashSet(
|
||||
Iterables.concat(
|
||||
Lists.transform(
|
||||
|
@ -86,7 +77,8 @@ public class SpatialDimensionRowFormatter
|
|||
);
|
||||
}
|
||||
|
||||
public InputRow formatRow(final InputRow row)
|
||||
@Override
|
||||
public InputRow apply(final InputRow row)
|
||||
{
|
||||
final Map<String, List<String>> spatialLookup = Maps.newHashMap();
|
||||
|
||||
|
@ -110,7 +102,7 @@ public class SpatialDimensionRowFormatter
|
|||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return !spatialDimNames.contains(input) && !spatialPartialDimNames.contains(input);
|
||||
return !spatialDimensionMap.containsKey(input) && !spatialPartialDimNames.contains(input);
|
||||
}
|
||||
}
|
||||
)
|
||||
|
@ -173,32 +165,32 @@ public class SpatialDimensionRowFormatter
|
|||
}
|
||||
};
|
||||
|
||||
if (!spatialPartialDimNames.isEmpty()) {
|
||||
for (SpatialDimensionSchema spatialDimension : spatialDimensions) {
|
||||
List<String> spatialDimVals = Lists.newArrayList();
|
||||
for (Map.Entry<String, SpatialDimensionSchema> entry : spatialDimensionMap.entrySet()) {
|
||||
final String spatialDimName = entry.getKey();
|
||||
final SpatialDimensionSchema spatialDim = entry.getValue();
|
||||
|
||||
for (String partialSpatialDim : spatialDimension.getDims()) {
|
||||
List<String> dimVals = row.getDimension(partialSpatialDim);
|
||||
if (isSpatialDimValsValid(dimVals)) {
|
||||
spatialDimVals.addAll(dimVals);
|
||||
}
|
||||
}
|
||||
|
||||
if (spatialDimVals.size() == spatialPartialDimNames.size()) {
|
||||
spatialLookup.put(spatialDimension.getDimName(), Arrays.asList(JOINER.join(spatialDimVals)));
|
||||
finalDims.add(spatialDimension.getDimName());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (String spatialDimName : spatialDimNames) {
|
||||
List<String> dimVals = row.getDimension(spatialDimName);
|
||||
List<String> dimVals = row.getDimension(spatialDimName);
|
||||
if (dimVals != null && !dimVals.isEmpty()) {
|
||||
if (dimVals.size() != 1) {
|
||||
throw new ISE("Cannot have a spatial dimension value with size[%d]", dimVals.size());
|
||||
throw new ISE("Spatial dimension value must be in an array!");
|
||||
}
|
||||
if (isJoinedSpatialDimValValid(dimVals.get(0))) {
|
||||
spatialLookup.put(spatialDimName, dimVals);
|
||||
finalDims.add(spatialDimName);
|
||||
}
|
||||
} else {
|
||||
List<String> spatialDimVals = Lists.newArrayList();
|
||||
for (String dim : spatialDim.getDims()) {
|
||||
List<String> partialDimVals = row.getDimension(dim);
|
||||
if (isSpatialDimValsValid(partialDimVals)) {
|
||||
spatialDimVals.addAll(partialDimVals);
|
||||
}
|
||||
}
|
||||
|
||||
if (spatialDimVals.size() == spatialDim.getDims().size()) {
|
||||
spatialLookup.put(spatialDimName, Arrays.asList(JOINER.join(spatialDimVals)));
|
||||
finalDims.add(spatialDimName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -43,13 +43,7 @@ import java.nio.channels.WritableByteChannel;
|
|||
*/
|
||||
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
public static DictionaryEncodedColumnPartSerde createDeserializer(
|
||||
boolean singleValued
|
||||
)
|
||||
{
|
||||
return new DictionaryEncodedColumnPartSerde();
|
||||
}
|
||||
private final boolean isSingleValued;
|
||||
|
||||
private final GenericIndexed<String> dictionary;
|
||||
private final VSizeIndexedInts singleValuedColumn;
|
||||
|
@ -67,6 +61,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
ImmutableRTree spatialIndex
|
||||
)
|
||||
{
|
||||
this.isSingleValued = multiValCol == null;
|
||||
this.dictionary = dictionary;
|
||||
this.singleValuedColumn = singleValCol;
|
||||
this.multiValuedColumn = multiValCol;
|
||||
|
@ -89,20 +84,25 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
this.size = size;
|
||||
}
|
||||
|
||||
private DictionaryEncodedColumnPartSerde()
|
||||
@JsonCreator
|
||||
public DictionaryEncodedColumnPartSerde(
|
||||
@JsonProperty("isSingleValued") boolean isSingleValued
|
||||
)
|
||||
{
|
||||
dictionary = null;
|
||||
singleValuedColumn = null;
|
||||
multiValuedColumn = null;
|
||||
bitmaps = null;
|
||||
spatialIndex = null;
|
||||
size = 0;
|
||||
this.isSingleValued = isSingleValued;
|
||||
|
||||
this.dictionary = null;
|
||||
this.singleValuedColumn = null;
|
||||
this.multiValuedColumn = null;
|
||||
this.bitmaps = null;
|
||||
this.spatialIndex = null;
|
||||
this.size = 0;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
private boolean isSingleValued()
|
||||
{
|
||||
return singleValuedColumn != null;
|
||||
return isSingleValued;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -114,14 +114,26 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
@Override
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued() ? 0x0 : 0x1)}));
|
||||
dictionary.writeToChannel(channel);
|
||||
if (isSingleValued()) {
|
||||
singleValuedColumn.writeToChannel(channel);
|
||||
} else {
|
||||
multiValuedColumn.writeToChannel(channel);
|
||||
channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued ? 0x0 : 0x1)}));
|
||||
|
||||
if (dictionary != null) {
|
||||
dictionary.writeToChannel(channel);
|
||||
}
|
||||
bitmaps.writeToChannel(channel);
|
||||
|
||||
if (isSingleValued()) {
|
||||
if (singleValuedColumn != null) {
|
||||
singleValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
} else {
|
||||
if (multiValuedColumn != null) {
|
||||
multiValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
}
|
||||
|
||||
if (bitmaps != null) {
|
||||
bitmaps.writeToChannel(channel);
|
||||
}
|
||||
|
||||
if (spatialIndex != null) {
|
||||
ByteBufferSerializer.writeToChannel(spatialIndex, IndexedRTree.objectStrategy, channel);
|
||||
}
|
||||
|
|
|
@ -194,9 +194,10 @@ public class QueryRunnerTestHelper
|
|||
)
|
||||
throws IOException
|
||||
{
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(false);
|
||||
final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex();
|
||||
final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex();
|
||||
final IncrementalIndex rtIndexOffheap = TestIndex.getIncrementalTestIndex(true);
|
||||
return Arrays.asList(
|
||||
new Object[][]{
|
||||
{
|
||||
|
@ -207,6 +208,9 @@ public class QueryRunnerTestHelper
|
|||
},
|
||||
{
|
||||
makeQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex))
|
||||
},
|
||||
{
|
||||
makeQueryRunner(factory, new IncrementalIndexSegment(rtIndexOffheap, segmentId))
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -218,9 +222,11 @@ public class QueryRunnerTestHelper
|
|||
)
|
||||
throws IOException
|
||||
{
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex();
|
||||
final IncrementalIndex rtIndex = TestIndex.getIncrementalTestIndex(false);
|
||||
final QueryableIndex mMappedTestIndex = TestIndex.getMMappedTestIndex();
|
||||
final QueryableIndex mergedRealtimeIndex = TestIndex.mergedRealtimeIndex();
|
||||
final IncrementalIndex rtIndexOffheap = TestIndex.getIncrementalTestIndex(true);
|
||||
|
||||
return Arrays.asList(
|
||||
new Object[][]{
|
||||
{
|
||||
|
@ -231,6 +237,9 @@ public class QueryRunnerTestHelper
|
|||
},
|
||||
{
|
||||
makeUnionQueryRunner(factory, new QueryableIndexSegment(segmentId, mergedRealtimeIndex))
|
||||
},
|
||||
{
|
||||
makeUnionQueryRunner(factory, new IncrementalIndexSegment(rtIndexOffheap, segmentId))
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -26,11 +26,10 @@ public class TestQueryRunners
|
|||
@Override
|
||||
public ByteBuffer get()
|
||||
{
|
||||
return ByteBuffer.allocate(1024 * 10);
|
||||
return ByteBuffer.allocate(1024 * 1024 * 10);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
public static final TopNQueryConfig topNConfig = new TopNQueryConfig();
|
||||
|
||||
public static StupidPool<ByteBuffer> getPool()
|
||||
|
|
|
@ -42,6 +42,7 @@ import io.druid.query.Query;
|
|||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
||||
|
@ -125,7 +126,8 @@ public class GroupByQueryRunnerTest
|
|||
engine,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
|
||||
configSupplier,
|
||||
new GroupByQueryQueryToolChest(configSupplier, mapper, engine)
|
||||
new GroupByQueryQueryToolChest(configSupplier,mapper, engine, TestQueryRunners.pool),
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
|
||||
GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig()
|
||||
|
@ -145,7 +147,8 @@ public class GroupByQueryRunnerTest
|
|||
singleThreadEngine,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
|
||||
singleThreadedConfigSupplier,
|
||||
new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine)
|
||||
new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine, pool),
|
||||
pool
|
||||
);
|
||||
|
||||
|
||||
|
@ -833,13 +836,9 @@ public class GroupByQueryRunnerTest
|
|||
}
|
||||
)
|
||||
);
|
||||
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(
|
||||
configSupplier,
|
||||
new DefaultObjectMapper(),
|
||||
engine
|
||||
).mergeResults(runner);
|
||||
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier,new DefaultObjectMapper(), engine, TestQueryRunners.pool).mergeResults(runner);
|
||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||
}
|
||||
|
||||
|
@ -888,12 +887,9 @@ public class GroupByQueryRunnerTest
|
|||
)
|
||||
);
|
||||
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(
|
||||
configSupplier,
|
||||
new DefaultObjectMapper(),
|
||||
engine
|
||||
).mergeResults(runner);
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine, TestQueryRunners.pool).mergeResults(runner);
|
||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -942,11 +938,7 @@ public class GroupByQueryRunnerTest
|
|||
)
|
||||
);
|
||||
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(
|
||||
configSupplier,
|
||||
new DefaultObjectMapper(),
|
||||
engine
|
||||
).mergeResults(runner);
|
||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine, TestQueryRunners.pool).mergeResults(runner);
|
||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,7 @@ import io.druid.query.Query;
|
|||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.timeseries.TimeseriesQuery;
|
||||
import io.druid.query.timeseries.TimeseriesQueryRunnerTest;
|
||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||
|
@ -76,7 +77,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
|
|||
engine,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
|
||||
configSupplier,
|
||||
new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine)
|
||||
new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine, TestQueryRunners.pool),
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
|
||||
final Collection<?> objects = QueryRunnerTestHelper.makeQueryRunners(factory);
|
||||
|
|
|
@ -50,7 +50,7 @@ public class SegmentAnalyzerTest
|
|||
public void testIncrementalDoesNotWork() throws Exception
|
||||
{
|
||||
final List<SegmentAnalysis> results = getSegmentAnalysises(
|
||||
new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), null)
|
||||
new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(false), null)
|
||||
);
|
||||
|
||||
Assert.assertEquals(0, results.size());
|
||||
|
@ -99,7 +99,7 @@ public class SegmentAnalyzerTest
|
|||
{
|
||||
final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
|
||||
(QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(new QueryConfig()),
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
), index
|
||||
);
|
||||
|
|
|
@ -49,7 +49,7 @@ public class SegmentMetadataQueryTest
|
|||
@SuppressWarnings("unchecked")
|
||||
private final QueryRunner runner = makeQueryRunner(
|
||||
new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(new QueryConfig()),
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER)
|
||||
);
|
||||
private ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
|
|
@ -116,10 +116,10 @@ public class SearchQueryRunnerTest
|
|||
Map<String, Set<String>> expectedResults = new HashMap<String, Set<String>>();
|
||||
expectedResults.put(
|
||||
QueryRunnerTestHelper.qualityDimension, new HashSet<String>(
|
||||
Arrays.asList(
|
||||
"automotive", "mezzanine", "travel", "health", "entertainment"
|
||||
Arrays.asList(
|
||||
"automotive", "mezzanine", "travel", "health", "entertainment"
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
checkSearchQuery(
|
||||
|
@ -367,6 +367,24 @@ public class SearchQueryRunnerTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testSearchNonExistingDimension()
|
||||
{
|
||||
Map<String, Set<String>> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
|
||||
|
||||
checkSearchQuery(
|
||||
Druids.newSearchQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(QueryRunnerTestHelper.allGran)
|
||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||
.dimensions("does_not_exist")
|
||||
.query("a")
|
||||
.build(),
|
||||
expectedResults
|
||||
);
|
||||
}
|
||||
|
||||
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
|
||||
{
|
||||
HashMap<String,List> context = new HashMap<String, List>();
|
||||
|
|
|
@ -33,6 +33,7 @@ import io.druid.query.QueryRunner;
|
|||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.segment.IncrementalIndexSegment;
|
||||
|
@ -52,7 +53,8 @@ public class TimeseriesQueryRunnerBonusTest
|
|||
public void testOneRowAtATime() throws Exception
|
||||
{
|
||||
final IncrementalIndex oneRowIndex = new IncrementalIndex(
|
||||
new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{}
|
||||
new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{},
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
|
||||
List<Result<TimeseriesResultValue>> results;
|
||||
|
|
|
@ -54,6 +54,7 @@ import io.druid.query.topn.TopNResultValue;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.segment;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexAdapter;
|
||||
|
@ -45,14 +46,18 @@ public class EmptyIndexTest
|
|||
}
|
||||
tmpDir.deleteOnExit();
|
||||
|
||||
IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||
IncrementalIndex emptyIndex = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0], TestQueryRunners.pool);
|
||||
IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex);
|
||||
IndexMerger.merge(Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir);
|
||||
IndexMaker.merge(
|
||||
Lists.<IndexableAdapter>newArrayList(emptyIndexAdapter),
|
||||
new AggregatorFactory[0],
|
||||
tmpDir
|
||||
);
|
||||
|
||||
QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir);
|
||||
|
||||
Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions()));
|
||||
Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyQueryableIndex.getColumnNames()));
|
||||
Assert.assertEquals("getDimensionNames", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions()));
|
||||
Assert.assertEquals("getMetricNames", 0, Iterables.size(emptyQueryableIndex.getColumnNames()));
|
||||
Assert.assertEquals("getDataInterval", new Interval("2012-08-01/P3D"), emptyQueryableIndex.getDataInterval());
|
||||
Assert.assertEquals("getReadOnlyTimestamps", 0, emptyQueryableIndex.getTimeColumn().getLength());
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.io.Files;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.data.IncrementalIndexTest;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
|
@ -37,7 +38,7 @@ import java.util.Arrays;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class IndexMergerTest
|
||||
public class IndexMakerTest
|
||||
{
|
||||
@Test
|
||||
public void testPersistCaseInsensitive() throws Exception
|
||||
|
@ -48,7 +49,7 @@ public class IndexMergerTest
|
|||
|
||||
final File tempDir = Files.createTempDir();
|
||||
try {
|
||||
QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir));
|
||||
QueryableIndex index = IndexIO.loadIndex(IndexMaker.persist(toPersist, tempDir));
|
||||
|
||||
Assert.assertEquals(2, index.getTimeColumn().getLength());
|
||||
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions()));
|
||||
|
@ -65,7 +66,7 @@ public class IndexMergerTest
|
|||
final long timestamp = System.currentTimeMillis();
|
||||
IncrementalIndex toPersist1 = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp);
|
||||
|
||||
IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||
IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool);
|
||||
|
||||
toPersist2.add(
|
||||
new MapBasedInputRow(
|
||||
|
@ -87,20 +88,24 @@ public class IndexMergerTest
|
|||
final File tempDir2 = Files.createTempDir();
|
||||
final File mergedDir = Files.createTempDir();
|
||||
try {
|
||||
QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1));
|
||||
QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tempDir1));
|
||||
|
||||
Assert.assertEquals(2, index1.getTimeColumn().getLength());
|
||||
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions()));
|
||||
Assert.assertEquals(2, index1.getColumnNames().size());
|
||||
|
||||
QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2));
|
||||
QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist2, tempDir2));
|
||||
|
||||
Assert.assertEquals(2, index2.getTimeColumn().getLength());
|
||||
Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions()));
|
||||
Assert.assertEquals(2, index2.getColumnNames().size());
|
||||
|
||||
QueryableIndex merged = IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir)
|
||||
IndexMaker.mergeQueryableIndex(
|
||||
Arrays.asList(index1, index2),
|
||||
new AggregatorFactory[]{},
|
||||
mergedDir
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(3, merged.getTimeColumn().getLength());
|
||||
|
@ -117,8 +122,8 @@ public class IndexMergerTest
|
|||
@Test
|
||||
public void testPersistEmptyColumn() throws Exception
|
||||
{
|
||||
final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||
final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||
final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool);
|
||||
final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}, TestQueryRunners.pool);
|
||||
final File tmpDir1 = Files.createTempDir();
|
||||
final File tmpDir2 = Files.createTempDir();
|
||||
final File tmpDir3 = Files.createTempDir();
|
||||
|
@ -140,10 +145,10 @@ public class IndexMergerTest
|
|||
)
|
||||
);
|
||||
|
||||
final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1));
|
||||
final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2));
|
||||
final QueryableIndex index1 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir1));
|
||||
final QueryableIndex index2 = IndexIO.loadIndex(IndexMaker.persist(toPersist1, tmpDir2));
|
||||
final QueryableIndex merged = IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3)
|
||||
IndexMaker.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3)
|
||||
);
|
||||
|
||||
Assert.assertEquals(1, index1.getTimeColumn().getLength());
|
||||
|
@ -154,7 +159,8 @@ public class IndexMergerTest
|
|||
|
||||
Assert.assertEquals(1, merged.getTimeColumn().getLength());
|
||||
Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions()));
|
||||
} finally {
|
||||
}
|
||||
finally {
|
||||
FileUtils.deleteQuietly(tmpDir1);
|
||||
FileUtils.deleteQuietly(tmpDir2);
|
||||
FileUtils.deleteQuietly(tmpDir3);
|
|
@ -33,6 +33,7 @@ import com.metamx.common.logger.Logger;
|
|||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
|
@ -133,7 +134,7 @@ public class SchemalessIndex
|
|||
final long timestamp = new DateTime(event.get(TIMESTAMP)).getMillis();
|
||||
|
||||
if (theIndex == null) {
|
||||
theIndex = new IncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS);
|
||||
theIndex = new IncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS, TestQueryRunners.pool);
|
||||
}
|
||||
|
||||
final List<String> dims = Lists.newArrayList();
|
||||
|
@ -178,11 +179,11 @@ public class SchemalessIndex
|
|||
mergedFile.mkdirs();
|
||||
mergedFile.deleteOnExit();
|
||||
|
||||
IndexMerger.persist(top, topFile);
|
||||
IndexMerger.persist(bottom, bottomFile);
|
||||
IndexMaker.persist(top, topFile);
|
||||
IndexMaker.persist(bottom, bottomFile);
|
||||
|
||||
mergedIndex = io.druid.segment.IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(
|
||||
IndexMaker.mergeQueryableIndex(
|
||||
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile
|
||||
)
|
||||
);
|
||||
|
@ -224,7 +225,7 @@ public class SchemalessIndex
|
|||
mergedFile.deleteOnExit();
|
||||
|
||||
QueryableIndex index = IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(
|
||||
IndexMaker.mergeQueryableIndex(
|
||||
Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile
|
||||
)
|
||||
);
|
||||
|
@ -261,7 +262,7 @@ public class SchemalessIndex
|
|||
}
|
||||
|
||||
QueryableIndex index = IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile)
|
||||
IndexMaker.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile)
|
||||
);
|
||||
|
||||
return index;
|
||||
|
@ -330,7 +331,7 @@ public class SchemalessIndex
|
|||
}
|
||||
|
||||
final IncrementalIndex rowIndex = new IncrementalIndex(
|
||||
timestamp, QueryGranularity.MINUTE, METRIC_AGGS
|
||||
timestamp, QueryGranularity.MINUTE, METRIC_AGGS, TestQueryRunners.pool
|
||||
);
|
||||
|
||||
rowIndex.add(
|
||||
|
@ -342,7 +343,7 @@ public class SchemalessIndex
|
|||
tmpFile.mkdirs();
|
||||
tmpFile.deleteOnExit();
|
||||
|
||||
IndexMerger.persist(rowIndex, tmpFile);
|
||||
IndexMaker.persist(rowIndex, tmpFile);
|
||||
rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile));
|
||||
}
|
||||
}
|
||||
|
@ -360,7 +361,7 @@ public class SchemalessIndex
|
|||
log.info("Realtime loading index file[%s]", filename);
|
||||
|
||||
final IncrementalIndex retVal = new IncrementalIndex(
|
||||
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs
|
||||
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs, TestQueryRunners.pool
|
||||
);
|
||||
|
||||
try {
|
||||
|
@ -402,7 +403,7 @@ public class SchemalessIndex
|
|||
theFile.mkdirs();
|
||||
theFile.deleteOnExit();
|
||||
filesToMap.add(theFile);
|
||||
IndexMerger.persist(index, theFile);
|
||||
IndexMaker.persist(index, theFile);
|
||||
}
|
||||
|
||||
return filesToMap;
|
||||
|
@ -462,7 +463,7 @@ public class SchemalessIndex
|
|||
);
|
||||
}
|
||||
|
||||
return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile));
|
||||
return IndexIO.loadIndex(IndexMaker.append(adapters, mergedFile));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
|
@ -481,7 +482,7 @@ public class SchemalessIndex
|
|||
List<File> filesToMap = makeFilesToMap(tmpFile, files);
|
||||
|
||||
return IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(
|
||||
IndexMaker.mergeQueryableIndex(
|
||||
Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
filesToMap,
|
||||
|
|
|
@ -1174,6 +1174,15 @@ public class SchemalessTestFull
|
|||
new DateTime("2011-01-12T00:00:00.000Z"),
|
||||
new TopNResultValue(
|
||||
Arrays.<Map<String, Object>>asList(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("provider", "")
|
||||
.put("rows", 6L)
|
||||
.put("index", 400.0D)
|
||||
.put("addRowsIndexConstant", 407.0D)
|
||||
.put("uniques", 0.0)
|
||||
.put("maxIndex", 100.0)
|
||||
.put("minIndex", 0.0)
|
||||
.build(),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("provider", "spot")
|
||||
.put("rows", 4L)
|
||||
|
@ -1183,15 +1192,6 @@ public class SchemalessTestFull
|
|||
.put("maxIndex", 100.0)
|
||||
.put("minIndex", 100.0)
|
||||
.build(),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("provider", "")
|
||||
.put("rows", 3L)
|
||||
.put("index", 200.0D)
|
||||
.put("addRowsIndexConstant", 204.0D)
|
||||
.put("uniques", 0.0)
|
||||
.put("maxIndex", 100.0)
|
||||
.put("minIndex", 0.0)
|
||||
.build(),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("provider", "total_market")
|
||||
.put("rows", 2L)
|
||||
|
@ -1386,7 +1386,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " timeseries ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
|
@ -1420,7 +1420,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered timeseries ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||
|
@ -1453,7 +1453,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " topN ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TopNResultValue>>newArrayList()
|
||||
|
@ -1487,7 +1487,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered topN ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TopNResultValue>>newArrayList()
|
||||
|
@ -1505,7 +1505,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " search ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<SearchResultValue>>newArrayList()
|
||||
|
@ -1524,7 +1524,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " filtered search ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<SearchResultValue>>newArrayList()
|
||||
|
@ -1543,7 +1543,7 @@ public class SchemalessTestFull
|
|||
.build();
|
||||
|
||||
failMsg += " timeBoundary ";
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||
Iterable<Result<TimeBoundaryResultValue>> actualResults = Sequences.toList(
|
||||
runner.run(query, context),
|
||||
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||
|
|
|
@ -31,11 +31,14 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.segment.incremental.OffheapIncrementalIndex;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -51,12 +54,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
*/
|
||||
public class TestIndex
|
||||
{
|
||||
private static final Logger log = new Logger(TestIndex.class);
|
||||
|
||||
private static IncrementalIndex realtimeIndex = null;
|
||||
private static QueryableIndex mmappedIndex = null;
|
||||
private static QueryableIndex mergedRealtime = null;
|
||||
|
||||
public static final String[] COLUMNS = new String[]{
|
||||
"ts",
|
||||
"provider",
|
||||
|
@ -68,6 +65,7 @@ public class TestIndex
|
|||
};
|
||||
public static final String[] DIMENSIONS = new String[]{"provider", "quALIty", "plAcEmEnT", "pLacementish"};
|
||||
public static final String[] METRICS = new String[]{"iNdEx"};
|
||||
private static final Logger log = new Logger(TestIndex.class);
|
||||
private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z");
|
||||
private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
|
||||
new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]),
|
||||
|
@ -80,7 +78,11 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
public static IncrementalIndex getIncrementalTestIndex()
|
||||
private static IncrementalIndex realtimeIndex = null;
|
||||
private static QueryableIndex mmappedIndex = null;
|
||||
private static QueryableIndex mergedRealtime = null;
|
||||
|
||||
public static IncrementalIndex getIncrementalTestIndex(boolean useOffheap)
|
||||
{
|
||||
synchronized (log) {
|
||||
if (realtimeIndex != null) {
|
||||
|
@ -88,7 +90,7 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
return realtimeIndex = makeRealtimeIndex("druid.sample.tsv");
|
||||
return realtimeIndex = makeRealtimeIndex("druid.sample.tsv", useOffheap);
|
||||
}
|
||||
|
||||
public static QueryableIndex getMMappedTestIndex()
|
||||
|
@ -99,7 +101,7 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
IncrementalIndex incrementalIndex = getIncrementalTestIndex();
|
||||
IncrementalIndex incrementalIndex = getIncrementalTestIndex(false);
|
||||
mmappedIndex = persistRealtimeAndLoadMMapped(incrementalIndex);
|
||||
|
||||
return mmappedIndex;
|
||||
|
@ -113,8 +115,8 @@ public class TestIndex
|
|||
}
|
||||
|
||||
try {
|
||||
IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top");
|
||||
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom");
|
||||
IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top", false);
|
||||
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom", false);
|
||||
|
||||
File tmpFile = File.createTempFile("yay", "who");
|
||||
tmpFile.delete();
|
||||
|
@ -130,11 +132,11 @@ public class TestIndex
|
|||
mergedFile.mkdirs();
|
||||
mergedFile.deleteOnExit();
|
||||
|
||||
IndexMerger.persist(top, DATA_INTERVAL, topFile);
|
||||
IndexMerger.persist(bottom, DATA_INTERVAL, bottomFile);
|
||||
IndexMaker.persist(top, DATA_INTERVAL, topFile);
|
||||
IndexMaker.persist(bottom, DATA_INTERVAL, bottomFile);
|
||||
|
||||
mergedRealtime = IndexIO.loadIndex(
|
||||
IndexMerger.mergeQueryableIndex(
|
||||
IndexMaker.mergeQueryableIndex(
|
||||
Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)),
|
||||
METRIC_AGGS,
|
||||
mergedFile
|
||||
|
@ -149,14 +151,27 @@ public class TestIndex
|
|||
}
|
||||
}
|
||||
|
||||
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename)
|
||||
private static IncrementalIndex makeRealtimeIndex(final String resourceFilename, final boolean useOffheap)
|
||||
{
|
||||
final URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename);
|
||||
log.info("Realtime loading index file[%s]", resource);
|
||||
|
||||
final IncrementalIndex retVal = new IncrementalIndex(
|
||||
new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.NONE, METRIC_AGGS
|
||||
);
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis())
|
||||
.withQueryGranularity(QueryGranularity.NONE)
|
||||
.withMetrics(METRIC_AGGS)
|
||||
.build();
|
||||
final IncrementalIndex retVal;
|
||||
if (useOffheap) {
|
||||
retVal = new OffheapIncrementalIndex(
|
||||
schema,
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
} else {
|
||||
retVal = new IncrementalIndex(
|
||||
schema,
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
}
|
||||
|
||||
final AtomicLong startTime = new AtomicLong();
|
||||
int lineCount;
|
||||
|
@ -228,7 +243,7 @@ public class TestIndex
|
|||
someTmpFile.mkdirs();
|
||||
someTmpFile.deleteOnExit();
|
||||
|
||||
IndexMerger.persist(index, someTmpFile);
|
||||
IndexMaker.persist(index, someTmpFile);
|
||||
return IndexIO.loadIndex(someTmpFile);
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.TestQueryRunners;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
|
@ -45,7 +46,10 @@ public class IncrementalIndexTest
|
|||
|
||||
public static IncrementalIndex createCaseInsensitiveIndex(long timestamp)
|
||||
{
|
||||
IncrementalIndex index = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{});
|
||||
IncrementalIndex index = new IncrementalIndex(
|
||||
0L, QueryGranularity.NONE, new AggregatorFactory[]{},
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
|
||||
index.add(
|
||||
new MapBasedInputRow(
|
||||
|
@ -105,7 +109,8 @@ public class IncrementalIndexTest
|
|||
final IncrementalIndex index = new IncrementalIndex(
|
||||
0L,
|
||||
QueryGranularity.NONE,
|
||||
new AggregatorFactory[]{new CountAggregatorFactory("count")}
|
||||
new AggregatorFactory[]{new CountAggregatorFactory("count")},
|
||||
TestQueryRunners.pool
|
||||
);
|
||||
final int threadCount = 10;
|
||||
final int elementsPerThread = 200;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue