mirror of https://github.com/apache/druid.git
Merge branch 'master' into igalDruid
This commit is contained in:
commit
805e80c24d
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Yielder;
|
||||
import com.metamx.common.guava.Yielders;
|
||||
|
@ -70,7 +71,7 @@ public class OrderedMergeSequence<T> implements Sequence<T>
|
|||
return yielder.get();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(yielder);
|
||||
CloseQuietly.close(yielder);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -46,14 +46,9 @@ public class SerializerUtils
|
|||
|
||||
public void writeString(OutputSupplier<? extends OutputStream> supplier, String name) throws IOException
|
||||
{
|
||||
OutputStream out = null;
|
||||
try {
|
||||
out = supplier.getOutput();
|
||||
try (OutputStream out = supplier.getOutput()) {
|
||||
writeString(out, name);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
}
|
||||
|
||||
public void writeString(WritableByteChannel out, String name) throws IOException
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.server;
|
||||
package io.druid.common.utils;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
|
@ -154,6 +154,7 @@ Druid storage nodes maintain information about segments they have already downlo
|
|||
|--------|-----------|-------|
|
||||
|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) |
|
||||
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|
||||
|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|5 minutes|
|
||||
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
|
||||
|
||||
### Jetty Server Module
|
||||
|
|
|
@ -69,10 +69,9 @@ A sample worker setup spec is shown below:
|
|||
"keyName":"keyName"
|
||||
},
|
||||
"userData":{
|
||||
"classType":"galaxy",
|
||||
"env":"druid",
|
||||
"version":"druid_version",
|
||||
"type":"sample_cluster/worker"
|
||||
"impl":"string",
|
||||
"data":"version=:VERSION:",
|
||||
"versionReplacementString":":VERSION:"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
@ -81,8 +80,8 @@ Issuing a GET request at the same URL will return the current worker setup spec
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none|
|
||||
|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be druid.indexer.runner.minWorkerVersion.|none|
|
||||
|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0|
|
||||
|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0|
|
||||
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|
||||
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|
||||
|`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required|
|
||||
|`userData`|A JSON object that describes how to configure new nodes. Currently, only EC2 is supported. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional|
|
||||
|
|
|
@ -22,6 +22,7 @@ Additional peon configs include:
|
|||
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|/tmp/persistent/tasks|
|
||||
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|
||||
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|
||||
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0|
|
||||
|`druid.indexer.task.chathandler.type`|Choices are "noop" and "announce". Certain tasks will use service discovery to announce an HTTP endpoint that events can be posted to.|noop|
|
||||
|
||||
If the peon is running in remote mode, there must be an overlord up and running. Running peons in remote mode require the following configurations:
|
||||
|
|
|
@ -153,7 +153,7 @@ druid.indexer.logs.s3Bucket=#{LOGS_BUCKET}
|
|||
druid.indexer.logs.s3Prefix=prod/logs/v1
|
||||
|
||||
# Dedicate more resources to peons
|
||||
druid.indexer.runner.javaOpts=-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
|
||||
druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
|
||||
druid.indexer.runner.taskDir=/mnt/persistent/task/
|
||||
druid.indexer.task.taskDir=/mnt/persistent/task/
|
||||
druid.indexer.task.chathandler.type=announce
|
||||
|
|
|
@ -69,6 +69,20 @@ You should see a bunch of files:
|
|||
* run_example_client.sh
|
||||
* LICENSE, config, examples, lib directories
|
||||
|
||||
Setting up Zookeeper
|
||||
--------------------
|
||||
|
||||
Before we get started, we need to start Apache Zookeeper.
|
||||
|
||||
```bash
|
||||
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
|
||||
tar xzf zookeeper-3.4.5.tar.gz
|
||||
cd zookeeper-3.4.5
|
||||
cp conf/zoo_sample.cfg conf/zoo.cfg
|
||||
./bin/zkServer.sh start
|
||||
cd ..
|
||||
```
|
||||
|
||||
Running Example Scripts
|
||||
-----------------------
|
||||
|
||||
|
|
|
@ -85,28 +85,23 @@ config/overlord/runtime.properties
|
|||
The configurations for the overlord node are as follows:
|
||||
|
||||
```bash
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
druid.host=localhost
|
||||
druid.port=8087
|
||||
druid.service=overlord
|
||||
|
||||
-Ddruid.host=localhost
|
||||
-Ddruid.port=8080
|
||||
-Ddruid.service=overlord
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
-Ddruid.zk.service.host=localhost
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
|
||||
|
||||
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
|
||||
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
-Ddruid.db.connector.user=druid
|
||||
-Ddruid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.selectors.indexing.serviceName=overlord
|
||||
-Ddruid.indexer.queue.startDelay=PT0M
|
||||
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
|
||||
druid.selectors.indexing.serviceName=overlord
|
||||
druid.indexer.queue.startDelay=PT0M
|
||||
druid.indexer.runner.javaOpts="-server -Xmx256m"
|
||||
druid.indexer.fork.property.druid.processing.numThreads=1
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=100000000
|
||||
```
|
||||
|
||||
If you are interested in reading more about these configurations, see [here](Indexing-Service.html).
|
||||
|
|
|
@ -48,7 +48,7 @@ CREATE database druid;
|
|||
#### Setting up Zookeeper
|
||||
|
||||
```bash
|
||||
curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
|
||||
curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
|
||||
tar xzf zookeeper-3.4.5.tar.gz
|
||||
cd zookeeper-3.4.5
|
||||
cp conf/zoo_sample.cfg conf/zoo.cfg
|
||||
|
@ -120,7 +120,7 @@ druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
|||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.coordinator.startDelay=PT60s
|
||||
druid.coordinator.startDelay=PT70s
|
||||
```
|
||||
|
||||
To start the coordinator node:
|
||||
|
|
|
@ -1,22 +1,17 @@
|
|||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
druid.host=localhost
|
||||
druid.port=8087
|
||||
druid.service=overlord
|
||||
|
||||
-Ddruid.host=localhost
|
||||
-Ddruid.port=8080
|
||||
-Ddruid.service=overlord
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
-Ddruid.zk.service.host=localhost
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
|
||||
|
||||
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"]
|
||||
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
-Ddruid.db.connector.user=druid
|
||||
-Ddruid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.selectors.indexing.serviceName=overlord
|
||||
-Ddruid.indexer.queue.startDelay=PT0M
|
||||
-Ddruid.indexer.runner.javaOpts="-server -Xmx256m"
|
||||
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000
|
||||
druid.selectors.indexing.serviceName=overlord
|
||||
druid.indexer.queue.startDelay=PT0M
|
||||
druid.indexer.runner.javaOpts="-server -Xmx256m"
|
||||
druid.indexer.fork.property.druid.processing.numThreads=1
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=100000000
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.storage.hdfs;
|
|||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.SegmentLoadingException;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -52,22 +53,17 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
|
|||
|
||||
final FileSystem fs = checkPathAndGetFilesystem(path);
|
||||
|
||||
FSDataInputStream in = null;
|
||||
try {
|
||||
if (path.getName().endsWith(".zip")) {
|
||||
in = fs.open(path);
|
||||
CompressionUtils.unzip(in, dir);
|
||||
in.close();
|
||||
if (path.getName().endsWith(".zip")) {
|
||||
try {
|
||||
try (FSDataInputStream in = fs.open(path)) {
|
||||
CompressionUtils.unzip(in, dir);
|
||||
}
|
||||
}
|
||||
else {
|
||||
throw new SegmentLoadingException("Unknown file type[%s]", path);
|
||||
catch (IOException e) {
|
||||
throw new SegmentLoadingException(e, "Some IOException");
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new SegmentLoadingException(e, "Some IOException");
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
} else {
|
||||
throw new SegmentLoadingException("Unknown file type[%s]", path);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,7 +81,8 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller
|
|||
}
|
||||
}
|
||||
|
||||
private Path getPath(DataSegment segment) {
|
||||
private Path getPath(DataSegment segment)
|
||||
{
|
||||
return new Path(String.valueOf(segment.getLoadSpec().get("path")));
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.io.ByteStreams;
|
|||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.OutputSupplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.segment.SegmentUtils;
|
||||
import io.druid.segment.loading.DataSegmentPusher;
|
||||
|
@ -78,17 +79,10 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
|
|||
|
||||
fs.mkdirs(outFile.getParent());
|
||||
log.info("Compressing files from[%s] to [%s]", inDir, outFile);
|
||||
FSDataOutputStream out = null;
|
||||
|
||||
long size;
|
||||
try {
|
||||
out = fs.create(outFile);
|
||||
|
||||
try (FSDataOutputStream out = fs.create(outFile)) {
|
||||
size = CompressionUtils.zip(inDir, out);
|
||||
|
||||
out.close();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
|
||||
return createDescriptorFile(
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -44,7 +44,9 @@ import io.druid.guice.JsonConfigProvider;
|
|||
import io.druid.guice.annotations.Self;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.path.PathSpec;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.indexing.granularity.GranularitySpec;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -81,7 +83,7 @@ public class HadoopDruidIndexerConfig
|
|||
|
||||
static {
|
||||
injector = Initialization.makeInjectorWithModules(
|
||||
Initialization.makeStartupInjector(),
|
||||
GuiceInjectors.makeStartupInjector(),
|
||||
ImmutableList.<Object>of(
|
||||
new Module()
|
||||
{
|
||||
|
@ -166,12 +168,14 @@ public class HadoopDruidIndexerConfig
|
|||
|
||||
private volatile HadoopIngestionSpec schema;
|
||||
private volatile PathSpec pathSpec;
|
||||
private volatile ColumnConfig columnConfig;
|
||||
|
||||
@JsonCreator
|
||||
public HadoopDruidIndexerConfig(
|
||||
final @JsonProperty("schema") HadoopIngestionSpec schema
|
||||
)
|
||||
{
|
||||
this.columnConfig = columnConfig;
|
||||
this.schema = schema;
|
||||
this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class);
|
||||
}
|
||||
|
@ -182,6 +186,11 @@ public class HadoopDruidIndexerConfig
|
|||
return schema;
|
||||
}
|
||||
|
||||
public ColumnConfig getColumnConfig()
|
||||
{
|
||||
return columnConfig;
|
||||
}
|
||||
|
||||
public String getDataSource()
|
||||
{
|
||||
return schema.getDataSchema().getDataSource();
|
||||
|
|
|
@ -31,6 +31,7 @@ import com.google.common.io.Closeables;
|
|||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
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;
|
||||
|
@ -425,7 +426,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
if (caughtException == null) {
|
||||
Closeables.close(out, false);
|
||||
} else {
|
||||
Closeables.closeQuietly(out);
|
||||
CloseQuietly.close(out);
|
||||
throw Throwables.propagate(caughtException);
|
||||
}
|
||||
}
|
||||
|
@ -605,7 +606,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
}
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
CloseQuietly.close(in);
|
||||
}
|
||||
out.closeEntry();
|
||||
context.progress();
|
||||
|
|
|
@ -45,7 +45,6 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath()
|
||||
{
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -49,7 +49,6 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
/**
|
||||
|
@ -187,7 +186,8 @@ public class TaskToolbox
|
|||
return retVal;
|
||||
}
|
||||
|
||||
public void pushSegments(Iterable<DataSegment> segments) throws IOException {
|
||||
public void pushSegments(Iterable<DataSegment> segments) throws IOException
|
||||
{
|
||||
// Request segment pushes for each set
|
||||
final Multimap<Interval, DataSegment> segmentMultimap = Multimaps.index(
|
||||
segments,
|
||||
|
|
|
@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
|
|||
import io.druid.indexing.common.config.TaskConfig;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.loading.DataSegmentArchiver;
|
||||
import io.druid.segment.loading.DataSegmentKiller;
|
||||
import io.druid.segment.loading.DataSegmentMover;
|
||||
|
|
|
@ -21,11 +21,17 @@ package io.druid.indexing.common.config;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
|
||||
public class TaskConfig
|
||||
{
|
||||
public static List<String> DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of(
|
||||
"org.apache.hadoop:hadoop-client:2.3.0"
|
||||
);
|
||||
|
||||
@JsonProperty
|
||||
private final String baseDir;
|
||||
|
||||
|
@ -38,40 +44,57 @@ public class TaskConfig
|
|||
@JsonProperty
|
||||
private final int defaultRowFlushBoundary;
|
||||
|
||||
@JsonProperty
|
||||
private final List<String> defaultHadoopCoordinates;
|
||||
|
||||
@JsonCreator
|
||||
public TaskConfig(
|
||||
@JsonProperty("baseDir") String baseDir,
|
||||
@JsonProperty("baseTaskDir") String baseTaskDir,
|
||||
@JsonProperty("hadoopWorkingPath") String hadoopWorkingPath,
|
||||
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary
|
||||
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary,
|
||||
@JsonProperty("defaultHadoopCoordinates") List<String> defaultHadoopCoordinates
|
||||
)
|
||||
{
|
||||
this.baseDir = baseDir == null ? "/tmp" : baseDir;
|
||||
this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task"));
|
||||
this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing");
|
||||
this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary;
|
||||
this.defaultHadoopCoordinates = defaultHadoopCoordinates == null
|
||||
? DEFAULT_DEFAULT_HADOOP_COORDINATES
|
||||
: defaultHadoopCoordinates;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getBaseDir()
|
||||
{
|
||||
return baseDir;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public File getBaseTaskDir()
|
||||
{
|
||||
return baseTaskDir;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getHadoopWorkingPath()
|
||||
{
|
||||
return hadoopWorkingPath;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getDefaultRowFlushBoundary()
|
||||
{
|
||||
return defaultRowFlushBoundary;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<String> getDefaultHadoopCoordinates()
|
||||
{
|
||||
return defaultHadoopCoordinates;
|
||||
}
|
||||
|
||||
private String defaultDir(String configParameter, final String defaultVal)
|
||||
{
|
||||
if (configParameter == null) {
|
||||
|
@ -80,4 +103,4 @@ public class TaskConfig
|
|||
|
||||
return configParameter;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.api.client.util.Lists;
|
|||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
|
@ -41,8 +42,9 @@ import io.druid.indexing.common.TaskToolbox;
|
|||
import io.druid.indexing.common.actions.LockAcquireAction;
|
||||
import io.druid.indexing.common.actions.LockTryAcquireAction;
|
||||
import io.druid.indexing.common.actions.TaskActionClient;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.server.initialization.ExtensionsConfig;
|
||||
import io.druid.guice.ExtensionsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.tesla.aether.internal.DefaultTeslaAether;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -62,11 +64,9 @@ public class HadoopIndexTask extends AbstractTask
|
|||
private static final ExtensionsConfig extensionsConfig;
|
||||
|
||||
static {
|
||||
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
|
||||
extensionsConfig = GuiceInjectors.makeStartupInjector().getInstance(ExtensionsConfig.class);
|
||||
}
|
||||
|
||||
public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0";
|
||||
|
||||
private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config)
|
||||
{
|
||||
if (spec != null) {
|
||||
|
@ -115,9 +115,14 @@ public class HadoopIndexTask extends AbstractTask
|
|||
Preconditions.checkArgument(this.spec.getTuningConfig().getWorkingPath() == null, "workingPath must be absent");
|
||||
Preconditions.checkArgument(this.spec.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent");
|
||||
|
||||
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.<String>asList(
|
||||
hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates
|
||||
) : hadoopDependencyCoordinates;
|
||||
if (hadoopDependencyCoordinates != null) {
|
||||
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates;
|
||||
} else if (hadoopCoordinates != null) {
|
||||
this.hadoopDependencyCoordinates = ImmutableList.of(hadoopCoordinates);
|
||||
} else {
|
||||
// Will be defaulted to something at runtime, based on taskConfig.
|
||||
this.hadoopDependencyCoordinates = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -158,6 +163,10 @@ public class HadoopIndexTask extends AbstractTask
|
|||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final List<String> finalHadoopDependencyCoordinates = hadoopDependencyCoordinates != null
|
||||
? hadoopDependencyCoordinates
|
||||
: toolbox.getConfig().getDefaultHadoopCoordinates();
|
||||
|
||||
final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig);
|
||||
|
||||
final List<URL> extensionURLs = Lists.newArrayList();
|
||||
|
@ -174,7 +183,7 @@ public class HadoopIndexTask extends AbstractTask
|
|||
final List<URL> driverURLs = Lists.newArrayList();
|
||||
driverURLs.addAll(nonHadoopURLs);
|
||||
// put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts
|
||||
for (String hadoopDependencyCoordinate : hadoopDependencyCoordinates) {
|
||||
for (String hadoopDependencyCoordinate : finalHadoopDependencyCoordinates) {
|
||||
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
|
||||
aetherClient, hadoopDependencyCoordinate
|
||||
);
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
|
||||
package io.druid.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.parsers.ParseException;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
|
@ -43,9 +44,10 @@ import io.druid.query.QueryRunner;
|
|||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import io.druid.segment.indexing.RealtimeIOConfig;
|
||||
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import io.druid.segment.realtime.FireDepartment;
|
||||
import io.druid.segment.realtime.FireDepartmentConfig;
|
||||
|
@ -353,7 +355,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
}
|
||||
}
|
||||
catch (FormattedException e) {
|
||||
catch (ParseException e) {
|
||||
log.warn(e, "unparseable line");
|
||||
fireDepartment.getMetrics().incrementUnparseable();
|
||||
}
|
||||
|
@ -375,7 +377,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
log.makeAlert(e, "Failed to finish realtime task").emit();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(firehose);
|
||||
CloseQuietly.close(firehose);
|
||||
toolbox.getMonitorScheduler().removeMonitor(metricsMonitor);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
|
|
@ -585,7 +585,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
|
||||
// Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running
|
||||
// on a worker - this avoids overflowing a worker with tasks
|
||||
Stopwatch timeoutStopwatch = new Stopwatch();
|
||||
Stopwatch timeoutStopwatch = Stopwatch.createUnstarted();
|
||||
timeoutStopwatch.start();
|
||||
synchronized (statusLock) {
|
||||
while (!isWorkerRunningTask(theWorker, task.getId())) {
|
||||
|
|
|
@ -29,19 +29,14 @@ import com.amazonaws.services.ec2.model.Reservation;
|
|||
import com.amazonaws.services.ec2.model.RunInstancesRequest;
|
||||
import com.amazonaws.services.ec2.model.RunInstancesResult;
|
||||
import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.indexing.overlord.setup.EC2NodeData;
|
||||
import io.druid.indexing.overlord.setup.GalaxyUserData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -50,20 +45,17 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
|
|||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class);
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final AmazonEC2 amazonEC2Client;
|
||||
private final SimpleResourceManagementConfig config;
|
||||
private final Supplier<WorkerSetupData> workerSetupDataRef;
|
||||
|
||||
@Inject
|
||||
public EC2AutoScalingStrategy(
|
||||
@Json ObjectMapper jsonMapper,
|
||||
AmazonEC2 amazonEC2Client,
|
||||
SimpleResourceManagementConfig config,
|
||||
Supplier<WorkerSetupData> workerSetupDataRef
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.amazonEC2Client = amazonEC2Client;
|
||||
this.config = config;
|
||||
this.workerSetupDataRef = workerSetupDataRef;
|
||||
|
@ -73,15 +65,21 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
|
|||
public AutoScalingData provision()
|
||||
{
|
||||
try {
|
||||
WorkerSetupData setupData = workerSetupDataRef.get();
|
||||
EC2NodeData workerConfig = setupData.getNodeData();
|
||||
final WorkerSetupData setupData = workerSetupDataRef.get();
|
||||
final EC2NodeData workerConfig = setupData.getNodeData();
|
||||
final String userDataBase64;
|
||||
|
||||
GalaxyUserData userData = setupData.getUserData();
|
||||
if (config.getWorkerVersion() != null) {
|
||||
userData = userData.withVersion(config.getWorkerVersion());
|
||||
if (setupData.getUserData() == null) {
|
||||
userDataBase64 = null;
|
||||
} else {
|
||||
if (config.getWorkerVersion() == null) {
|
||||
userDataBase64 = setupData.getUserData().getUserDataBase64();
|
||||
} else {
|
||||
userDataBase64 = setupData.getUserData().withVersion(config.getWorkerVersion()).getUserDataBase64();
|
||||
}
|
||||
}
|
||||
|
||||
RunInstancesResult result = amazonEC2Client.runInstances(
|
||||
final RunInstancesResult result = amazonEC2Client.runInstances(
|
||||
new RunInstancesRequest(
|
||||
workerConfig.getAmiId(),
|
||||
workerConfig.getMinInstances(),
|
||||
|
@ -91,16 +89,10 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
|
|||
.withSecurityGroupIds(workerConfig.getSecurityGroupIds())
|
||||
.withPlacement(new Placement(setupData.getAvailabilityZone()))
|
||||
.withKeyName(workerConfig.getKeyName())
|
||||
.withUserData(
|
||||
Base64.encodeBase64String(
|
||||
jsonMapper.writeValueAsBytes(
|
||||
userData
|
||||
)
|
||||
)
|
||||
)
|
||||
.withUserData(userDataBase64)
|
||||
);
|
||||
|
||||
List<String> instanceIds = Lists.transform(
|
||||
final List<String> instanceIds = Lists.transform(
|
||||
result.getReservation().getInstances(),
|
||||
new Function<Instance, String>()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Represents any user data that may be needed to launch EC2 instances.
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "impl", defaultImpl = GalaxyEC2UserData.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "galaxy", value = GalaxyEC2UserData.class),
|
||||
@JsonSubTypes.Type(name = "string", value = StringEC2UserData.class)
|
||||
})
|
||||
public interface EC2UserData<T extends EC2UserData>
|
||||
{
|
||||
/**
|
||||
* Return a copy of this instance with a different worker version. If no changes are needed (possibly because the
|
||||
* user data does not depend on the worker version) then it is OK to return "this".
|
||||
*/
|
||||
public EC2UserData<T> withVersion(String version);
|
||||
|
||||
public String getUserDataBase64();
|
||||
}
|
|
@ -19,24 +19,32 @@
|
|||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class GalaxyUserData
|
||||
public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
|
||||
{
|
||||
public final String env;
|
||||
public final String version;
|
||||
public final String type;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final String env;
|
||||
private final String version;
|
||||
private final String type;
|
||||
|
||||
@JsonCreator
|
||||
public GalaxyUserData(
|
||||
public GalaxyEC2UserData(
|
||||
@JacksonInject @Json ObjectMapper jsonMapper,
|
||||
@JsonProperty("env") String env,
|
||||
@JsonProperty("version") String version,
|
||||
@JsonProperty("type") String type
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.env = env;
|
||||
this.version = version;
|
||||
this.type = type;
|
||||
|
@ -60,9 +68,21 @@ public class GalaxyUserData
|
|||
return type;
|
||||
}
|
||||
|
||||
public GalaxyUserData withVersion(String ver)
|
||||
@Override
|
||||
public GalaxyEC2UserData withVersion(String ver)
|
||||
{
|
||||
return new GalaxyUserData(env, ver, type);
|
||||
return new GalaxyEC2UserData(jsonMapper, env, ver, type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUserDataBase64()
|
||||
{
|
||||
try {
|
||||
return Base64.encodeBase64String(jsonMapper.writeValueAsBytes(this));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* 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.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.api.client.util.Charsets;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
public class StringEC2UserData implements EC2UserData<StringEC2UserData>
|
||||
{
|
||||
private final String data;
|
||||
private final String versionReplacementString;
|
||||
private final String version;
|
||||
|
||||
@JsonCreator
|
||||
public StringEC2UserData(
|
||||
@JsonProperty("data") String data,
|
||||
@JsonProperty("versionReplacementString") String versionReplacementString,
|
||||
@JsonProperty("version") String version
|
||||
)
|
||||
{
|
||||
this.data = data;
|
||||
this.versionReplacementString = versionReplacementString;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getData()
|
||||
{
|
||||
return data;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersionReplacementString()
|
||||
{
|
||||
return versionReplacementString;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StringEC2UserData withVersion(final String _version)
|
||||
{
|
||||
return new StringEC2UserData(data, versionReplacementString, _version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUserDataBase64()
|
||||
{
|
||||
final String finalData;
|
||||
if (versionReplacementString != null && version != null) {
|
||||
finalData = data.replace(versionReplacementString, version);
|
||||
} else {
|
||||
finalData = data;
|
||||
}
|
||||
return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "StringEC2UserData{" +
|
||||
"data='" + data + '\'' +
|
||||
", versionReplacementString='" + versionReplacementString + '\'' +
|
||||
", version='" + version + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -33,7 +33,7 @@ public class WorkerSetupData
|
|||
private final int maxNumWorkers;
|
||||
private final String availabilityZone;
|
||||
private final EC2NodeData nodeData;
|
||||
private final GalaxyUserData userData;
|
||||
private final EC2UserData userData;
|
||||
|
||||
@JsonCreator
|
||||
public WorkerSetupData(
|
||||
|
@ -42,7 +42,7 @@ public class WorkerSetupData
|
|||
@JsonProperty("maxNumWorkers") int maxNumWorkers,
|
||||
@JsonProperty("availabilityZone") String availabilityZone,
|
||||
@JsonProperty("nodeData") EC2NodeData nodeData,
|
||||
@JsonProperty("userData") GalaxyUserData userData
|
||||
@JsonProperty("userData") EC2UserData userData
|
||||
)
|
||||
{
|
||||
this.minVersion = minVersion;
|
||||
|
@ -84,7 +84,7 @@ public class WorkerSetupData
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public GalaxyUserData getUserData()
|
||||
public EC2UserData getUserData()
|
||||
{
|
||||
return userData;
|
||||
}
|
||||
|
|
|
@ -19,19 +19,51 @@
|
|||
|
||||
package io.druid.indexing.common;
|
||||
|
||||
import com.fasterxml.jackson.databind.BeanProperty;
|
||||
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Stopwatch;
|
||||
import com.metamx.common.ISE;
|
||||
import io.druid.guice.ServerModule;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class TestUtils
|
||||
{
|
||||
public static final ObjectMapper MAPPER = new DefaultObjectMapper();
|
||||
|
||||
static {
|
||||
final List<? extends Module> list = new ServerModule().getJacksonModules();
|
||||
for (Module module : list) {
|
||||
MAPPER.registerModule(module);
|
||||
}
|
||||
MAPPER.setInjectableValues(
|
||||
new InjectableValues()
|
||||
{
|
||||
@Override
|
||||
public Object findInjectableValue(
|
||||
Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance
|
||||
)
|
||||
{
|
||||
if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) {
|
||||
return TestUtils.MAPPER;
|
||||
}
|
||||
throw new ISE("No Injectable value found");
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public static boolean conditionValid(IndexingServiceCondition condition)
|
||||
{
|
||||
try {
|
||||
Stopwatch stopwatch = new Stopwatch();
|
||||
Stopwatch stopwatch = Stopwatch.createUnstarted();
|
||||
stopwatch.start();
|
||||
while (!condition.isValid()) {
|
||||
Thread.sleep(100);
|
||||
|
|
|
@ -48,6 +48,8 @@ import java.io.File;
|
|||
|
||||
public class TaskSerdeTest
|
||||
{
|
||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
|
||||
@Test
|
||||
public void testIndexTaskSerde() throws Exception
|
||||
{
|
||||
|
@ -68,7 +70,6 @@ public class TaskSerdeTest
|
|||
-1
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
}
|
||||
|
@ -102,7 +103,6 @@ public class TaskSerdeTest
|
|||
)
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -131,7 +131,6 @@ public class TaskSerdeTest
|
|||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -153,7 +152,6 @@ public class TaskSerdeTest
|
|||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -177,7 +175,6 @@ public class TaskSerdeTest
|
|||
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -208,7 +205,6 @@ public class TaskSerdeTest
|
|||
null
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -251,7 +247,6 @@ public class TaskSerdeTest
|
|||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -269,7 +264,6 @@ public class TaskSerdeTest
|
|||
@Test
|
||||
public void testDeleteTaskFromJson() throws Exception
|
||||
{
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final DeleteTask task = (DeleteTask) jsonMapper.readValue(
|
||||
"{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}",
|
||||
Task.class
|
||||
|
@ -300,7 +294,6 @@ public class TaskSerdeTest
|
|||
)
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -325,7 +318,6 @@ public class TaskSerdeTest
|
|||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -350,7 +342,6 @@ public class TaskSerdeTest
|
|||
new Interval("2010-01-01/P1D")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -375,7 +366,6 @@ public class TaskSerdeTest
|
|||
ImmutableMap.<String, Object>of("bucket", "hey", "baseKey", "what")
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
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
|
||||
|
@ -434,7 +424,6 @@ public class TaskSerdeTest
|
|||
null
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final String json = jsonMapper.writeValueAsString(task);
|
||||
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
|
@ -135,7 +136,7 @@ public class TaskLifecycleTest
|
|||
mdc = newMockMDC();
|
||||
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
|
||||
tb = new TaskToolboxFactory(
|
||||
new TaskConfig(tmp.toString(), null, null, 50000),
|
||||
new TaskConfig(tmp.toString(), null, null, 50000, null),
|
||||
tac,
|
||||
newMockEmitter(),
|
||||
new DataSegmentPusher()
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* 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.base.Charsets;
|
||||
import io.druid.indexing.common.TestUtils;
|
||||
import io.druid.indexing.overlord.setup.EC2UserData;
|
||||
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
|
||||
import io.druid.indexing.overlord.setup.StringEC2UserData;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class WorkerSetupDataTest
|
||||
{
|
||||
@Test
|
||||
public void testGalaxyEC2UserDataSerde() throws IOException
|
||||
{
|
||||
final String json = "{\"env\":\"druid\",\"version\":null,\"type\":\"typical\"}";
|
||||
final GalaxyEC2UserData userData = (GalaxyEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
|
||||
Assert.assertEquals("druid", userData.getEnv());
|
||||
Assert.assertEquals("typical", userData.getType());
|
||||
Assert.assertNull(userData.getVersion());
|
||||
Assert.assertEquals("1234", userData.withVersion("1234").getVersion());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStringEC2UserDataSerde() throws IOException
|
||||
{
|
||||
final String json = "{\"impl\":\"string\",\"data\":\"hey :ver:\",\"versionReplacementString\":\":ver:\",\"version\":\"1234\"}";
|
||||
final StringEC2UserData userData = (StringEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class);
|
||||
Assert.assertEquals("hey :ver:", userData.getData());
|
||||
Assert.assertEquals("1234", userData.getVersion());
|
||||
Assert.assertEquals(
|
||||
Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)),
|
||||
userData.getUserDataBase64()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)),
|
||||
userData.withVersion("xyz").getUserDataBase64()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -30,7 +30,7 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.common.guava.DSuppliers;
|
||||
import io.druid.indexing.overlord.setup.EC2NodeData;
|
||||
import io.druid.indexing.overlord.setup.GalaxyUserData;
|
||||
import io.druid.indexing.overlord.setup.GalaxyEC2UserData;
|
||||
import io.druid.indexing.overlord.setup.WorkerSetupData;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -75,7 +75,6 @@ public class EC2AutoScalingStrategyTest
|
|||
.withPrivateIpAddress(IP);
|
||||
|
||||
strategy = new EC2AutoScalingStrategy(
|
||||
new DefaultObjectMapper(),
|
||||
amazonEC2Client,
|
||||
new SimpleResourceManagementConfig().setWorkerPort(8080).setWorkerVersion(""),
|
||||
DSuppliers.of(workerSetupData)
|
||||
|
@ -101,7 +100,7 @@ public class EC2AutoScalingStrategyTest
|
|||
1,
|
||||
"",
|
||||
new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.<String>newArrayList(), "foo"),
|
||||
new GalaxyUserData("env", "version", "type")
|
||||
new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type")
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -20,11 +20,17 @@
|
|||
package io.druid.indexing.worker;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
|
||||
import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector;
|
||||
import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import io.druid.curator.PotentiallyGzippedCompressionProvider;
|
||||
import io.druid.indexing.common.IndexingServiceCondition;
|
||||
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||
|
@ -38,6 +44,7 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
|
|||
import io.druid.indexing.overlord.ThreadPoolTaskRunner;
|
||||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.loading.DataSegmentPuller;
|
||||
import io.druid.segment.loading.LocalDataSegmentPuller;
|
||||
import io.druid.segment.loading.OmniSegmentLoader;
|
||||
|
@ -61,6 +68,40 @@ import java.util.List;
|
|||
public class WorkerTaskMonitorTest
|
||||
{
|
||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
private static final Injector injector = Guice.createInjector(
|
||||
new com.google.inject.Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(ColumnConfig.class).toInstance(
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 1024 * 1024;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
static {
|
||||
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
|
||||
|
||||
jsonMapper.setInjectableValues(new GuiceInjectableValues(injector));
|
||||
jsonMapper.setAnnotationIntrospectors(
|
||||
new AnnotationIntrospectorPair(
|
||||
guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector()
|
||||
),
|
||||
new AnnotationIntrospectorPair(
|
||||
guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private static final Joiner joiner = Joiner.on("/");
|
||||
private static final String basePath = "/test/druid";
|
||||
private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath);
|
||||
|
@ -121,7 +162,7 @@ public class WorkerTaskMonitorTest
|
|||
workerCuratorCoordinator,
|
||||
new ThreadPoolTaskRunner(
|
||||
new TaskToolboxFactory(
|
||||
new TaskConfig(tmp.toString(), null, null, 0),
|
||||
new TaskConfig(tmp.toString(), null, null, 0, null),
|
||||
null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory(
|
||||
new OmniSegmentLoader(
|
||||
ImmutableMap.<String, DataSegmentPuller>of(
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
|
@ -115,7 +114,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputRow nextRow() throws FormattedException
|
||||
public InputRow nextRow()
|
||||
{
|
||||
final byte[] message = iter.next().message();
|
||||
|
||||
|
@ -123,15 +122,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory<ByteBufferInpu
|
|||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
return theParser.parse(ByteBuffer.wrap(message));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new FormattedException.Builder()
|
||||
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
|
||||
.withMessage(String.format("Error parsing[%s], got [%s]", ByteBuffer.wrap(message), e.toString()))
|
||||
.build();
|
||||
}
|
||||
return theParser.parse(ByteBuffer.wrap(message));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
|
@ -123,7 +122,7 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputRow nextRow() throws FormattedException
|
||||
public InputRow nextRow()
|
||||
{
|
||||
final Message message = iter.next().message();
|
||||
|
||||
|
@ -134,17 +133,9 @@ public class KafkaSevenFirehoseFactory implements FirehoseFactory<ByteBufferInpu
|
|||
return parseMessage(message);
|
||||
}
|
||||
|
||||
public InputRow parseMessage(Message message) throws FormattedException
|
||||
public InputRow parseMessage(Message message)
|
||||
{
|
||||
try {
|
||||
return theParser.parse(message.payload());
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new FormattedException.Builder()
|
||||
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW)
|
||||
.withMessage(String.format("Error parsing[%s], got [%s]", message.payload(), e.toString()))
|
||||
.build();
|
||||
}
|
||||
return theParser.parse(message.payload());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
31
pom.xml
31
pom.xml
|
@ -18,20 +18,19 @@
|
|||
~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<packaging>pom</packaging>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
<name>druid</name>
|
||||
<description>druid</description>
|
||||
<scm>
|
||||
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
|
||||
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
|
||||
<url>http://www.github.com/metamx/druid</url>
|
||||
<tag>druid-0.6.107-SNAPSHOT</tag>
|
||||
<tag>druid-0.6.117-SNAPSHOT</tag>
|
||||
</scm>
|
||||
|
||||
<prerequisites>
|
||||
|
@ -40,9 +39,9 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.25.6</metamx.java-util.version>
|
||||
<apache.curator.version>2.4.0</apache.curator.version>
|
||||
<druid.api.version>0.2.3</druid.api.version>
|
||||
<metamx.java-util.version>0.26.5</metamx.java-util.version>
|
||||
<apache.curator.version>2.5.0</apache.curator.version>
|
||||
<druid.api.version>0.2.4</druid.api.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
|
@ -199,22 +198,22 @@
|
|||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<version>14.0.1</version>
|
||||
<version>17.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
<version>4.0-beta4</version>
|
||||
<version>4.0-beta</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-servlet</artifactId>
|
||||
<version>4.0-beta4</version>
|
||||
<version>4.0-beta</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-multibindings</artifactId>
|
||||
<version>4.0-beta4</version>
|
||||
<version>4.0-beta</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.ibm.icu</groupId>
|
||||
|
@ -562,15 +561,7 @@
|
|||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-release-plugin</artifactId>
|
||||
<version>2.4.2</version>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.maven.scm</groupId>
|
||||
<artifactId>maven-scm-provider-gitexe</artifactId>
|
||||
<!-- This version is necessary for use with git version 1.8.5 and above -->
|
||||
<version>1.8.1</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<version>2.5</version>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</pluginManagement>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,10 +28,8 @@ import com.google.protobuf.ByteString;
|
|||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.DynamicMessage;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.JSONParseSpec;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.data.input.impl.ParseSpec;
|
||||
|
@ -94,7 +92,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputRow parse(ByteBuffer input) throws FormattedException
|
||||
public InputRow parse(ByteBuffer input)
|
||||
{
|
||||
// We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses
|
||||
// the DynamicMessage directly...
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.server.initialization;
|
||||
package io.druid.guice;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.server.initialization;
|
||||
package io.druid.guice;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
|
@ -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.guice;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.util.Modules;
|
||||
import io.druid.guice.ConfigModule;
|
||||
import io.druid.guice.DruidGuiceExtensions;
|
||||
import io.druid.guice.DruidSecondaryModule;
|
||||
import io.druid.guice.ExtensionsConfig;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.PropertiesModule;
|
||||
import io.druid.jackson.JacksonModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class GuiceInjectors
|
||||
{
|
||||
public static Injector makeStartupInjector()
|
||||
{
|
||||
return Guice.createInjector(
|
||||
new DruidGuiceExtensions(),
|
||||
new JacksonModule(),
|
||||
new PropertiesModule("runtime.properties"),
|
||||
new ConfigModule(),
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(DruidSecondaryModule.class);
|
||||
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public static Injector makeStartupInjectorWithModules(Iterable<Module> modules)
|
||||
{
|
||||
List<Module> theModules = Lists.newArrayList();
|
||||
theModules.add(new DruidGuiceExtensions());
|
||||
theModules.add(new JacksonModule());
|
||||
theModules.add(new PropertiesModule("runtime.properties"));
|
||||
theModules.add(new ConfigModule());
|
||||
theModules.add(
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(DruidSecondaryModule.class);
|
||||
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
|
||||
}
|
||||
}
|
||||
);
|
||||
for (Module theModule : modules) {
|
||||
theModules.add(theModule);
|
||||
}
|
||||
|
||||
|
||||
return Guice.createInjector(theModules);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* 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.guice;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Module;
|
||||
import com.metamx.common.ISE;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.guice.annotations.Smile;
|
||||
import io.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ModuleList
|
||||
{
|
||||
private final Injector baseInjector;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final ObjectMapper smileMapper;
|
||||
private final List<Module> modules;
|
||||
|
||||
public ModuleList(Injector baseInjector)
|
||||
{
|
||||
this.baseInjector = baseInjector;
|
||||
this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
|
||||
this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class));
|
||||
this.modules = Lists.newArrayList();
|
||||
}
|
||||
|
||||
public List<Module> getModules()
|
||||
{
|
||||
return Collections.unmodifiableList(modules);
|
||||
}
|
||||
|
||||
public void addModule(Object input)
|
||||
{
|
||||
if (input instanceof DruidModule) {
|
||||
baseInjector.injectMembers(input);
|
||||
modules.add(registerJacksonModules(((DruidModule) input)));
|
||||
} else if (input instanceof Module) {
|
||||
baseInjector.injectMembers(input);
|
||||
modules.add((Module) input);
|
||||
} else if (input instanceof Class) {
|
||||
if (DruidModule.class.isAssignableFrom((Class) input)) {
|
||||
modules.add(registerJacksonModules(baseInjector.getInstance((Class<? extends DruidModule>) input)));
|
||||
} else if (Module.class.isAssignableFrom((Class) input)) {
|
||||
modules.add(baseInjector.getInstance((Class<? extends Module>) input));
|
||||
return;
|
||||
} else {
|
||||
throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class);
|
||||
}
|
||||
} else {
|
||||
throw new ISE("Unknown module type[%s]", input.getClass());
|
||||
}
|
||||
}
|
||||
|
||||
public void addModules(Object... object)
|
||||
{
|
||||
for (Object o : object) {
|
||||
addModule(o);
|
||||
}
|
||||
}
|
||||
|
||||
private DruidModule registerJacksonModules(DruidModule module)
|
||||
{
|
||||
for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) {
|
||||
jsonMapper.registerModule(jacksonModule);
|
||||
smileMapper.registerModule(jacksonModule);
|
||||
}
|
||||
return module;
|
||||
}
|
||||
}
|
|
@ -17,13 +17,13 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.server.initialization;
|
||||
package io.druid.guice;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
|
@ -80,7 +80,7 @@ public class PropertiesModule implements Module
|
|||
log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up.");
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(stream);
|
||||
CloseQuietly.close(stream);
|
||||
}
|
||||
|
||||
binder.bind(Properties.class).toInstance(props);
|
|
@ -17,14 +17,13 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.server;
|
||||
package io.druid.query;
|
||||
|
||||
import com.metamx.common.concurrent.ExecutorServiceConfig;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class DruidProcessingConfig extends ExecutorServiceConfig
|
||||
public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig
|
||||
{
|
||||
@Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"})
|
||||
public int intermediateComputeSizeBytes()
|
||||
|
@ -39,4 +38,10 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig
|
|||
final int processors = Runtime.getRuntime().availableProcessors();
|
||||
return processors > 1 ? processors - 1 : processors;
|
||||
}
|
||||
|
||||
@Config(value = "${base_path}.columnCache.sizeBytes")
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 1024 * 1024;
|
||||
}
|
||||
}
|
|
@ -692,12 +692,14 @@ public class Druids
|
|||
{
|
||||
private DataSource dataSource;
|
||||
private QuerySegmentSpec querySegmentSpec;
|
||||
private String bound;
|
||||
private Map<String, Object> context;
|
||||
|
||||
public TimeBoundaryQueryBuilder()
|
||||
{
|
||||
dataSource = null;
|
||||
querySegmentSpec = null;
|
||||
bound = null;
|
||||
context = null;
|
||||
}
|
||||
|
||||
|
@ -706,6 +708,7 @@ public class Druids
|
|||
return new TimeBoundaryQuery(
|
||||
dataSource,
|
||||
querySegmentSpec,
|
||||
bound,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
@ -715,6 +718,7 @@ public class Druids
|
|||
return new TimeBoundaryQueryBuilder()
|
||||
.dataSource(builder.dataSource)
|
||||
.intervals(builder.querySegmentSpec)
|
||||
.bound(builder.bound)
|
||||
.context(builder.context);
|
||||
}
|
||||
|
||||
|
@ -748,6 +752,12 @@ public class Druids
|
|||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder bound(String b)
|
||||
{
|
||||
bound = b;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder context(Map<String, Object> c)
|
||||
{
|
||||
context = c;
|
||||
|
|
|
@ -88,7 +88,6 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
|
|||
@Override
|
||||
public Sequence<Row> run(final Query<Row> queryParam)
|
||||
{
|
||||
|
||||
final GroupByQuery query = (GroupByQuery) queryParam;
|
||||
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
|
||||
query,
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.ResourceClosingSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import io.druid.segment.ReferenceCountingSegment;
|
||||
|
@ -52,7 +52,7 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
|
|||
return new ResourceClosingSequence<T>(baseSequence, closeable);
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
Closeables.closeQuietly(closeable);
|
||||
CloseQuietly.close(closeable);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,9 +26,9 @@ import java.util.List;
|
|||
|
||||
/**
|
||||
* Processing related interface
|
||||
*
|
||||
* <p/>
|
||||
* An AggregatorFactory is an object that knows how to generate an Aggregator using a ColumnSelectorFactory.
|
||||
*
|
||||
* <p/>
|
||||
* This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects
|
||||
* without making any assumptions about how they are pulling values out of the base data. That is, the data is
|
||||
* provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how
|
||||
|
@ -37,7 +37,9 @@ import java.util.List;
|
|||
public interface AggregatorFactory
|
||||
{
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory);
|
||||
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory);
|
||||
|
||||
public Comparator getComparator();
|
||||
|
||||
/**
|
||||
|
@ -48,6 +50,7 @@ public interface AggregatorFactory
|
|||
*
|
||||
* @param lhs The left hand side of the combine
|
||||
* @param rhs The right hand side of the combine
|
||||
*
|
||||
* @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs
|
||||
*/
|
||||
public Object combine(Object lhs, Object rhs);
|
||||
|
@ -61,11 +64,19 @@ public interface AggregatorFactory
|
|||
*/
|
||||
public AggregatorFactory getCombiningFactory();
|
||||
|
||||
/**
|
||||
* Gets a list of all columns that this AggregatorFactory will scan
|
||||
*
|
||||
* @return AggregatorFactories for the columns to scan of the parent AggregatorFactory
|
||||
*/
|
||||
public List<AggregatorFactory> getRequiredColumns();
|
||||
|
||||
/**
|
||||
* A method that knows how to "deserialize" the object from whatever form it might have been put into
|
||||
* in order to transfer via JSON.
|
||||
*
|
||||
* @param object the object to deserialize
|
||||
*
|
||||
* @return the deserialized object
|
||||
*/
|
||||
public Object deserialize(Object object);
|
||||
|
@ -75,13 +86,17 @@ public interface AggregatorFactory
|
|||
* intermediate format than their final resultant output.
|
||||
*
|
||||
* @param object the object to be finalized
|
||||
*
|
||||
* @return the finalized value that should be returned for the initial query
|
||||
*/
|
||||
public Object finalizeComputation(Object object);
|
||||
|
||||
public String getName();
|
||||
|
||||
public List<String> requiredFields();
|
||||
|
||||
public byte[] getCacheKey();
|
||||
|
||||
public String getTypeName();
|
||||
|
||||
/**
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.primitives.Longs;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -76,6 +77,12 @@ public class CountAggregatorFactory implements AggregatorFactory
|
|||
return new LongSumAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new CountAggregatorFactory(name));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -136,12 +143,18 @@ public class CountAggregatorFactory implements AggregatorFactory
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
CountAggregatorFactory that = (CountAggregatorFactory) o;
|
||||
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) return false;
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -85,6 +85,12 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
|
|||
return new DoubleSumAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new DoubleSumAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -158,13 +164,21 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DoubleSumAggregatorFactory that = (DoubleSumAggregatorFactory) o;
|
||||
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false;
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) return false;
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -56,7 +56,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory
|
|||
|
||||
this.name = name;
|
||||
this.fieldName = fieldName;
|
||||
this.breaksList = (breaksList == null) ? Lists.<Float>newArrayList() :breaksList;
|
||||
this.breaksList = (breaksList == null) ? Lists.<Float>newArrayList() : breaksList;
|
||||
this.breaks = new float[this.breaksList.size()];
|
||||
for (int i = 0; i < this.breaksList.size(); ++i) {
|
||||
this.breaks[i] = this.breaksList.get(i);
|
||||
|
@ -100,6 +100,12 @@ public class HistogramAggregatorFactory implements AggregatorFactory
|
|||
return new HistogramAggregatorFactory(name, name, breaksList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new HistogramAggregatorFactory(fieldName, fieldName, breaksList));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -183,15 +189,27 @@ public class HistogramAggregatorFactory implements AggregatorFactory
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
HistogramAggregatorFactory that = (HistogramAggregatorFactory) o;
|
||||
|
||||
if (!Arrays.equals(breaks, that.breaks)) return false;
|
||||
if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) return false;
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false;
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) return false;
|
||||
if (!Arrays.equals(breaks, that.breaks)) {
|
||||
return false;
|
||||
}
|
||||
if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) {
|
||||
return false;
|
||||
}
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -140,6 +140,22 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
|
|||
return new JavaScriptAggregatorFactory(name, Lists.newArrayList(name), fnCombine, fnReset, fnCombine);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Lists.transform(
|
||||
fieldNames,
|
||||
new com.google.common.base.Function<String, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(String input)
|
||||
{
|
||||
return new JavaScriptAggregatorFactory(input, fieldNames, fnAggregate, fnReset, fnCombine);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
|
|
@ -31,11 +31,11 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
*/
|
||||
public class LongSumAggregatorFactory implements AggregatorFactory
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
|
||||
private final String fieldName;
|
||||
private final String name;
|
||||
|
||||
|
@ -85,6 +85,12 @@ public class LongSumAggregatorFactory implements AggregatorFactory
|
|||
return new LongSumAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new LongSumAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -154,13 +160,21 @@ public class LongSumAggregatorFactory implements AggregatorFactory
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
LongSumAggregatorFactory that = (LongSumAggregatorFactory) o;
|
||||
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false;
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) return false;
|
||||
if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -82,6 +82,12 @@ public class MaxAggregatorFactory implements AggregatorFactory
|
|||
return new MaxAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new MaxAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
|
|
@ -82,6 +82,12 @@ public class MinAggregatorFactory implements AggregatorFactory
|
|||
return new MinAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new MinAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
|
|
@ -65,6 +65,12 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory
|
|||
return baseAggregatorFactory.getCombiningFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return baseAggregatorFactory.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
|
|
@ -54,7 +54,7 @@ public class CardinalityAggregator implements Aggregator
|
|||
// nothing to add to hasher if size == 0, only handle size == 1 and size != 0 cases.
|
||||
if (size == 1) {
|
||||
final String value = selector.lookupName(row.get(0));
|
||||
hasher.putString(value != null ? value : NULL_STRING);
|
||||
hasher.putUnencodedChars(value != null ? value : NULL_STRING);
|
||||
} else if (size != 0) {
|
||||
final String[] values = new String[size];
|
||||
for (int i = 0; i < size; ++i) {
|
||||
|
@ -67,7 +67,7 @@ public class CardinalityAggregator implements Aggregator
|
|||
if (i != 0) {
|
||||
hasher.putChar(SEPARATOR);
|
||||
}
|
||||
hasher.putString(values[i]);
|
||||
hasher.putUnencodedChars(values[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ public class CardinalityAggregator implements Aggregator
|
|||
for (final DimensionSelector selector : selectors) {
|
||||
for (final Integer index : selector.getRow()) {
|
||||
final String value = selector.lookupName(index);
|
||||
collector.add(hashFn.hashString(value == null ? NULL_STRING : value).asBytes());
|
||||
collector.add(hashFn.hashUnencodedChars(value == null ? NULL_STRING : value).asBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,12 +32,14 @@ import io.druid.query.aggregation.AggregatorFactory;
|
|||
import io.druid.query.aggregation.Aggregators;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -142,7 +144,23 @@ public class CardinalityAggregatorFactory implements AggregatorFactory
|
|||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
return new CardinalityAggregatorFactory(name, fieldNames, byRow);
|
||||
return new HyperUniquesAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Lists.transform(
|
||||
fieldNames,
|
||||
new Function<String, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(String input)
|
||||
{
|
||||
return new CardinalityAggregatorFactory(input, fieldNames, byRow);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -73,12 +73,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
|||
return Aggregators.noopAggregator();
|
||||
}
|
||||
|
||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||
final Class classOfObject = selector.classOfObject();
|
||||
if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) {
|
||||
return new HyperUniquesAggregator(name, selector);
|
||||
}
|
||||
|
||||
throw new IAE(
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject()
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -91,12 +92,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
|||
return Aggregators.noopBufferAggregator();
|
||||
}
|
||||
|
||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||
final Class classOfObject = selector.classOfObject();
|
||||
if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) {
|
||||
return new HyperUniquesBufferAggregator(selector);
|
||||
}
|
||||
|
||||
throw new IAE(
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject()
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -131,6 +133,12 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
|||
return new HyperUniquesAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Arrays.<AggregatorFactory>asList(new HyperUniquesAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
|
|
@ -128,8 +128,9 @@ public class HyperUniquesSerde extends ComplexMetricSerde
|
|||
@Override
|
||||
public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes)
|
||||
{
|
||||
buffer.limit(buffer.position() + numBytes);
|
||||
return HyperLogLogCollector.makeCollector(buffer);
|
||||
final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
|
||||
readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes);
|
||||
return HyperLogLogCollector.makeCollector(readOnlyBuffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -72,7 +72,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
private final List<AggregatorFactory> aggregatorSpecs;
|
||||
private final List<PostAggregator> postAggregatorSpecs;
|
||||
|
||||
private final Function<Sequence<Row>, Sequence<Row>> orderByLimitFn;
|
||||
private final Function<Sequence<Row>, Sequence<Row>> limitFn;
|
||||
|
||||
@JsonCreator
|
||||
public GroupByQuery(
|
||||
|
@ -85,8 +85,9 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
@JsonProperty("postAggregations") List<PostAggregator> postAggregatorSpecs,
|
||||
@JsonProperty("having") HavingSpec havingSpec,
|
||||
@JsonProperty("limitSpec") LimitSpec limitSpec,
|
||||
@JsonProperty("orderBy") LimitSpec orderBySpec,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
@JsonProperty("context") Map<String, Object> context,
|
||||
// Backwards compatible
|
||||
@JsonProperty("orderBy") LimitSpec orderBySpec
|
||||
)
|
||||
{
|
||||
super(dataSource, querySegmentSpec, context);
|
||||
|
@ -129,7 +130,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
);
|
||||
}
|
||||
|
||||
orderByLimitFn = postProcFn;
|
||||
limitFn = postProcFn;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -146,7 +147,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
List<PostAggregator> postAggregatorSpecs,
|
||||
HavingSpec havingSpec,
|
||||
LimitSpec orderBySpec,
|
||||
Function<Sequence<Row>, Sequence<Row>> orderByLimitFn,
|
||||
Function<Sequence<Row>, Sequence<Row>> limitFn,
|
||||
Map<String, Object> context
|
||||
)
|
||||
{
|
||||
|
@ -159,7 +160,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
this.postAggregatorSpecs = postAggregatorSpecs;
|
||||
this.havingSpec = havingSpec;
|
||||
this.limitSpec = orderBySpec;
|
||||
this.orderByLimitFn = orderByLimitFn;
|
||||
this.limitFn = limitFn;
|
||||
}
|
||||
|
||||
@JsonProperty("filter")
|
||||
|
@ -199,7 +200,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public LimitSpec getOrderBy()
|
||||
public LimitSpec getLimitSpec()
|
||||
{
|
||||
return limitSpec;
|
||||
}
|
||||
|
@ -218,7 +219,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
|
||||
public Sequence<Row> applyLimit(Sequence<Row> results)
|
||||
{
|
||||
return orderByLimitFn.apply(results);
|
||||
return limitFn.apply(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -234,7 +235,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
postAggregatorSpecs,
|
||||
havingSpec,
|
||||
limitSpec,
|
||||
orderByLimitFn,
|
||||
limitFn,
|
||||
computeOverridenContext(contextOverride)
|
||||
);
|
||||
}
|
||||
|
@ -252,7 +253,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
postAggregatorSpecs,
|
||||
havingSpec,
|
||||
limitSpec,
|
||||
orderByLimitFn,
|
||||
limitFn,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
@ -270,7 +271,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
postAggregatorSpecs,
|
||||
havingSpec,
|
||||
limitSpec,
|
||||
orderByLimitFn,
|
||||
limitFn,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
@ -292,11 +293,25 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
private List<OrderByColumnSpec> orderByColumnSpecs = Lists.newArrayList();
|
||||
private int limit = Integer.MAX_VALUE;
|
||||
|
||||
private Builder()
|
||||
public Builder()
|
||||
{
|
||||
}
|
||||
|
||||
private Builder(Builder builder)
|
||||
public Builder(GroupByQuery query)
|
||||
{
|
||||
dataSource = query.getDataSource();
|
||||
querySegmentSpec = query.getQuerySegmentSpec();
|
||||
limitSpec = query.getLimitSpec();
|
||||
dimFilter = query.getDimFilter();
|
||||
granularity = query.getGranularity();
|
||||
dimensions = query.getDimensions();
|
||||
aggregatorSpecs = query.getAggregatorSpecs();
|
||||
postAggregatorSpecs = query.getPostAggregatorSpecs();
|
||||
havingSpec = query.getHavingSpec();
|
||||
context = query.getContext();
|
||||
}
|
||||
|
||||
public Builder(Builder builder)
|
||||
{
|
||||
dataSource = builder.dataSource;
|
||||
querySegmentSpec = builder.querySegmentSpec;
|
||||
|
@ -490,7 +505,11 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
{
|
||||
final LimitSpec theLimitSpec;
|
||||
if (limitSpec == null) {
|
||||
theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit);
|
||||
if (orderByColumnSpecs.isEmpty() && limit == Integer.MAX_VALUE) {
|
||||
theLimitSpec = new NoopLimitSpec();
|
||||
} else {
|
||||
theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit);
|
||||
}
|
||||
} else {
|
||||
theLimitSpec = limitSpec;
|
||||
}
|
||||
|
@ -504,9 +523,9 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
aggregatorSpecs,
|
||||
postAggregatorSpecs,
|
||||
havingSpec,
|
||||
null,
|
||||
theLimitSpec,
|
||||
context
|
||||
context,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -515,36 +534,57 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
public String toString()
|
||||
{
|
||||
return "GroupByQuery{" +
|
||||
"limitSpec=" + limitSpec +
|
||||
", dimFilter=" + dimFilter +
|
||||
", granularity=" + granularity +
|
||||
", dimensions=" + dimensions +
|
||||
", aggregatorSpecs=" + aggregatorSpecs +
|
||||
", postAggregatorSpecs=" + postAggregatorSpecs +
|
||||
", orderByLimitFn=" + orderByLimitFn +
|
||||
'}';
|
||||
"limitSpec=" + limitSpec +
|
||||
", dimFilter=" + dimFilter +
|
||||
", granularity=" + granularity +
|
||||
", dimensions=" + dimensions +
|
||||
", aggregatorSpecs=" + aggregatorSpecs +
|
||||
", postAggregatorSpecs=" + postAggregatorSpecs +
|
||||
", limitFn=" + limitFn +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (!super.equals(o)) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
GroupByQuery that = (GroupByQuery) o;
|
||||
|
||||
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null)
|
||||
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) {
|
||||
return false;
|
||||
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false;
|
||||
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false;
|
||||
if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false;
|
||||
if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false;
|
||||
if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false;
|
||||
if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null)
|
||||
}
|
||||
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) {
|
||||
return false;
|
||||
if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null)
|
||||
}
|
||||
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) {
|
||||
return false;
|
||||
}
|
||||
if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) {
|
||||
return false;
|
||||
}
|
||||
if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) {
|
||||
return false;
|
||||
}
|
||||
if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) {
|
||||
return false;
|
||||
}
|
||||
if (limitFn != null ? !limitFn.equals(that.limitFn) : that.limitFn != null) {
|
||||
return false;
|
||||
}
|
||||
if (postAggregatorSpecs != null
|
||||
? !postAggregatorSpecs.equals(that.postAggregatorSpecs)
|
||||
: that.postAggregatorSpecs != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -560,7 +600,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
|
||||
result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0);
|
||||
result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0);
|
||||
result = 31 * result + (orderByLimitFn != null ? orderByLimitFn.hashCode() : 0);
|
||||
result = 31 * result + (limitFn != null ? limitFn.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,12 +25,12 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.FunctionalIterator;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
|
@ -123,7 +123,7 @@ public class GroupByQueryEngine
|
|||
@Override
|
||||
public void cleanup(RowIterator iterFromMake)
|
||||
{
|
||||
Closeables.closeQuietly(iterFromMake);
|
||||
CloseQuietly.close(iterFromMake);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -135,7 +135,7 @@ public class GroupByQueryEngine
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(bufferHolder);
|
||||
CloseQuietly.close(bufferHolder);
|
||||
}
|
||||
}
|
||||
)
|
||||
|
|
|
@ -24,12 +24,14 @@ 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.data.input.MapBasedRow;
|
||||
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.IncrementalIndexSchema;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
@ -53,7 +55,7 @@ public class GroupByQueryHelper
|
|||
new Function<AggregatorFactory, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(@Nullable AggregatorFactory input)
|
||||
public AggregatorFactory apply(AggregatorFactory input)
|
||||
{
|
||||
return input.getCombiningFactory();
|
||||
}
|
||||
|
@ -64,7 +66,7 @@ public class GroupByQueryHelper
|
|||
new Function<DimensionSpec, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(@Nullable DimensionSpec input)
|
||||
public String apply(DimensionSpec input)
|
||||
{
|
||||
return input.getOutputName();
|
||||
}
|
||||
|
@ -83,14 +85,14 @@ public class GroupByQueryHelper
|
|||
@Override
|
||||
public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in)
|
||||
{
|
||||
if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions)) > config.getMaxResults()) {
|
||||
if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions), false) > config.getMaxResults()) {
|
||||
throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults());
|
||||
}
|
||||
|
||||
return accumulated;
|
||||
}
|
||||
};
|
||||
return new Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>>(index, accumulator);
|
||||
return new Pair<>(index, accumulator);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ 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;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.Pair;
|
||||
|
@ -44,11 +45,13 @@ import io.druid.query.QueryToolChest;
|
|||
import io.druid.query.SubqueryQueryRunner;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -59,7 +62,10 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
{
|
||||
};
|
||||
private static final String GROUP_BY_MERGE_KEY = "groupByMerge";
|
||||
private static final Map<String, Object> NO_MERGE_CONTEXT = ImmutableMap.<String, Object>of(GROUP_BY_MERGE_KEY, "false");
|
||||
private static final Map<String, Object> NO_MERGE_CONTEXT = ImmutableMap.<String, Object>of(
|
||||
GROUP_BY_MERGE_KEY,
|
||||
"false"
|
||||
);
|
||||
private final Supplier<GroupByQueryConfig> configSupplier;
|
||||
private GroupByQueryEngine engine; // For running the outer query around a subquery
|
||||
|
||||
|
@ -81,7 +87,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
@Override
|
||||
public Sequence<Row> run(Query<Row> input)
|
||||
{
|
||||
if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
||||
if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
||||
return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner);
|
||||
} else {
|
||||
return runner.run(input);
|
||||
|
@ -92,33 +98,45 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
|
||||
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner)
|
||||
{
|
||||
|
||||
Sequence<Row> result;
|
||||
|
||||
// If there's a subquery, merge subquery results and then apply the aggregator
|
||||
DataSource dataSource = query.getDataSource();
|
||||
final DataSource dataSource = query.getDataSource();
|
||||
if (dataSource instanceof QueryDataSource) {
|
||||
GroupByQuery subquery;
|
||||
try {
|
||||
subquery = (GroupByQuery) ((QueryDataSource) dataSource).getQuery();
|
||||
} catch (ClassCastException e) {
|
||||
}
|
||||
catch (ClassCastException e) {
|
||||
throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
|
||||
}
|
||||
Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner);
|
||||
IncrementalIndexStorageAdapter adapter
|
||||
= new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult));
|
||||
result = engine.process(query, adapter);
|
||||
final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner);
|
||||
final List<AggregatorFactory> aggs = Lists.newArrayList();
|
||||
for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
|
||||
aggs.addAll(aggregatorFactory.getRequiredColumns());
|
||||
}
|
||||
|
||||
// We need the inner incremental index to have all the columns required by the outer query
|
||||
final GroupByQuery innerQuery = new GroupByQuery.Builder(query)
|
||||
.setAggregatorSpecs(aggs)
|
||||
.setInterval(subquery.getIntervals())
|
||||
.setPostAggregatorSpecs(Lists.<PostAggregator>newArrayList())
|
||||
.build();
|
||||
|
||||
final GroupByQuery outerQuery = new GroupByQuery.Builder(query)
|
||||
.setLimitSpec(query.getLimitSpec().merge(subquery.getLimitSpec()))
|
||||
.build();
|
||||
|
||||
final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(
|
||||
makeIncrementalIndex(innerQuery, subqueryResult)
|
||||
);
|
||||
return outerQuery.applyLimit(engine.process(outerQuery, adapter));
|
||||
} else {
|
||||
result = runner.run(query);
|
||||
return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query))));
|
||||
}
|
||||
|
||||
return postAggregate(query, makeIncrementalIndex(query, result));
|
||||
}
|
||||
|
||||
|
||||
private Sequence<Row> postAggregate(final GroupByQuery query, IncrementalIndex index)
|
||||
{
|
||||
Sequence<Row> sequence = Sequences.map(
|
||||
return Sequences.map(
|
||||
Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())),
|
||||
new Function<Row, Row>()
|
||||
{
|
||||
|
@ -128,13 +146,12 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
final MapBasedRow row = (MapBasedRow) input;
|
||||
return new MapBasedRow(
|
||||
query.getGranularity()
|
||||
.toDateTime(row.getTimestampFromEpoch()),
|
||||
.toDateTime(row.getTimestampFromEpoch()),
|
||||
row.getEvent()
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
return query.applyLimit(sequence);
|
||||
}
|
||||
|
||||
private IncrementalIndex makeIncrementalIndex(GroupByQuery query, Sequence<Row> rows)
|
||||
|
@ -152,7 +169,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
@Override
|
||||
public Sequence<Row> mergeSequences(Sequence<Sequence<Row>> seqOfSequences)
|
||||
{
|
||||
return new ConcatSequence<Row>(seqOfSequences);
|
||||
return new ConcatSequence<>(seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -87,12 +87,17 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
|
||||
if (limit == Integer.MAX_VALUE) {
|
||||
return new SortingFn(ordering);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
return new TopNFunction(ordering, limit);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LimitSpec merge(LimitSpec other)
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
private Ordering<Row> makeComparator(
|
||||
List<DimensionSpec> dimensions, List<AggregatorFactory> aggs, List<PostAggregator> postAggs
|
||||
)
|
||||
|
@ -200,12 +205,18 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
LimitingFn that = (LimitingFn) o;
|
||||
|
||||
if (limit != that.limit) return false;
|
||||
if (limit != that.limit) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -232,12 +243,18 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SortingFn sortingFn = (SortingFn) o;
|
||||
|
||||
if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) return false;
|
||||
if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -273,13 +290,21 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
TopNFunction that = (TopNFunction) o;
|
||||
|
||||
if (limit != that.limit) return false;
|
||||
if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) return false;
|
||||
if (limit != that.limit) {
|
||||
return false;
|
||||
}
|
||||
if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -296,13 +321,21 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DefaultLimitSpec that = (DefaultLimitSpec) o;
|
||||
|
||||
if (limit != that.limit) return false;
|
||||
if (columns != null ? !columns.equals(that.columns) : that.columns != null) return false;
|
||||
if (limit != that.limit) {
|
||||
return false;
|
||||
}
|
||||
if (columns != null ? !columns.equals(that.columns) : that.columns != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -38,5 +38,11 @@ import java.util.List;
|
|||
})
|
||||
public interface LimitSpec
|
||||
{
|
||||
public Function<Sequence<Row>, Sequence<Row>> build(List<DimensionSpec> dimensions, List<AggregatorFactory> aggs, List<PostAggregator> postAggs);
|
||||
public Function<Sequence<Row>, Sequence<Row>> build(
|
||||
List<DimensionSpec> dimensions,
|
||||
List<AggregatorFactory> aggs,
|
||||
List<PostAggregator> postAggs
|
||||
);
|
||||
|
||||
public LimitSpec merge(LimitSpec other);
|
||||
}
|
||||
|
|
|
@ -41,6 +41,12 @@ public class NoopLimitSpec implements LimitSpec
|
|||
return Functions.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public LimitSpec merge(LimitSpec other)
|
||||
{
|
||||
return other;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.query.timeboundary;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -48,12 +49,16 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
);
|
||||
public static final String MAX_TIME = "maxTime";
|
||||
public static final String MIN_TIME = "minTime";
|
||||
|
||||
private static final byte CACHE_TYPE_ID = 0x0;
|
||||
|
||||
private final String bound;
|
||||
|
||||
@JsonCreator
|
||||
public TimeBoundaryQuery(
|
||||
@JsonProperty("dataSource") DataSource dataSource,
|
||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||
@JsonProperty("bound") String bound,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
)
|
||||
{
|
||||
|
@ -63,6 +68,8 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
: querySegmentSpec,
|
||||
context
|
||||
);
|
||||
|
||||
this.bound = bound == null ? "" : bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -77,12 +84,19 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return Query.TIME_BOUNDARY;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getBound()
|
||||
{
|
||||
return bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeBoundaryQuery withOverriddenContext(Map<String, Object> contextOverrides)
|
||||
{
|
||||
return new TimeBoundaryQuery(
|
||||
getDataSource(),
|
||||
getQuerySegmentSpec(),
|
||||
bound,
|
||||
computeOverridenContext(contextOverrides)
|
||||
);
|
||||
}
|
||||
|
@ -93,6 +107,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return new TimeBoundaryQuery(
|
||||
getDataSource(),
|
||||
spec,
|
||||
bound,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
@ -103,40 +118,33 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
return new TimeBoundaryQuery(
|
||||
dataSource,
|
||||
getQuerySegmentSpec(),
|
||||
bound,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return ByteBuffer.allocate(1)
|
||||
final byte[] boundBytes = bound.getBytes(Charsets.UTF_8);
|
||||
return ByteBuffer.allocate(1 + boundBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(boundBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "TimeBoundaryQuery{" +
|
||||
"dataSource='" + getDataSource() + '\'' +
|
||||
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||
", duration=" + getDuration() +
|
||||
'}';
|
||||
}
|
||||
|
||||
public Iterable<Result<TimeBoundaryResultValue>> buildResult(DateTime timestamp, DateTime min, DateTime max)
|
||||
{
|
||||
List<Result<TimeBoundaryResultValue>> results = Lists.newArrayList();
|
||||
Map<String, Object> result = Maps.newHashMap();
|
||||
|
||||
if (min != null) {
|
||||
result.put(TimeBoundaryQuery.MIN_TIME, min);
|
||||
result.put(MIN_TIME, min);
|
||||
}
|
||||
if (max != null) {
|
||||
result.put(TimeBoundaryQuery.MAX_TIME, max);
|
||||
result.put(MAX_TIME, max);
|
||||
}
|
||||
if (!result.isEmpty()) {
|
||||
results.add(new Result<TimeBoundaryResultValue>(timestamp, new TimeBoundaryResultValue(result)));
|
||||
results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result)));
|
||||
}
|
||||
|
||||
return results;
|
||||
|
@ -154,25 +162,74 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
TimeBoundaryResultValue val = result.getValue();
|
||||
|
||||
DateTime currMinTime = val.getMinTime();
|
||||
if (currMinTime.isBefore(min)) {
|
||||
if (currMinTime != null && currMinTime.isBefore(min)) {
|
||||
min = currMinTime;
|
||||
}
|
||||
DateTime currMaxTime = val.getMaxTime();
|
||||
if (currMaxTime.isAfter(max)) {
|
||||
if (currMaxTime != null && currMaxTime.isAfter(max)) {
|
||||
max = currMaxTime;
|
||||
}
|
||||
}
|
||||
|
||||
return Arrays.asList(
|
||||
new Result<TimeBoundaryResultValue>(
|
||||
min,
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
TimeBoundaryQuery.MIN_TIME, min,
|
||||
TimeBoundaryQuery.MAX_TIME, max
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
final DateTime ts;
|
||||
final DateTime minTime;
|
||||
final DateTime maxTime;
|
||||
|
||||
if (bound.equalsIgnoreCase(MIN_TIME)) {
|
||||
ts = min;
|
||||
minTime = min;
|
||||
maxTime = null;
|
||||
} else if (bound.equalsIgnoreCase(MAX_TIME)) {
|
||||
ts = max;
|
||||
minTime = null;
|
||||
maxTime = max;
|
||||
} else {
|
||||
ts = min;
|
||||
minTime = min;
|
||||
maxTime = max;
|
||||
}
|
||||
|
||||
return buildResult(ts, minTime, maxTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "TimeBoundaryQuery{" +
|
||||
"dataSource='" + getDataSource() + '\'' +
|
||||
", querySegmentSpec=" + getQuerySegmentSpec() +
|
||||
", duration=" + getDuration() +
|
||||
", bound" + bound +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
TimeBoundaryQuery that = (TimeBoundaryQuery) o;
|
||||
|
||||
if (!bound.equals(that.bound)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = super.hashCode();
|
||||
result = 31 * result + bound.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,8 +67,8 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
return segments;
|
||||
}
|
||||
|
||||
final T first = segments.get(0);
|
||||
final T second = segments.get(segments.size() - 1);
|
||||
final T min = segments.get(0);
|
||||
final T max = segments.get(segments.size() - 1);
|
||||
|
||||
return Lists.newArrayList(
|
||||
Iterables.filter(
|
||||
|
@ -78,8 +78,8 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
@Override
|
||||
public boolean apply(T input)
|
||||
{
|
||||
return input.getInterval().overlaps(first.getInterval()) || input.getInterval()
|
||||
.overlaps(second.getInterval());
|
||||
return (min != null && input.getInterval().overlaps(min.getInterval())) ||
|
||||
(max != null && input.getInterval().overlaps(max.getInterval()));
|
||||
}
|
||||
}
|
||||
)
|
||||
|
@ -111,7 +111,7 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
@Override
|
||||
public Sequence<Result<TimeBoundaryResultValue>> mergeSequences(Sequence<Sequence<Result<TimeBoundaryResultValue>>> seqOfSequences)
|
||||
{
|
||||
return new OrderedMergeSequence<Result<TimeBoundaryResultValue>>(getOrdering(), seqOfSequences);
|
||||
return new OrderedMergeSequence<>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -146,9 +146,9 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
public byte[] computeCacheKey(TimeBoundaryQuery query)
|
||||
{
|
||||
return ByteBuffer.allocate(2)
|
||||
.put(TIMEBOUNDARY_QUERY)
|
||||
.put(query.getCacheKey())
|
||||
.array();
|
||||
.put(TIMEBOUNDARY_QUERY)
|
||||
.put(query.getCacheKey())
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -177,11 +177,11 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
{
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Result<TimeBoundaryResultValue> apply(@Nullable Object input)
|
||||
public Result<TimeBoundaryResultValue> apply(Object input)
|
||||
{
|
||||
List<Object> result = (List<Object>) input;
|
||||
|
||||
return new Result<TimeBoundaryResultValue>(
|
||||
return new Result<>(
|
||||
new DateTime(result.get(0)),
|
||||
new TimeBoundaryResultValue(result.get(1))
|
||||
);
|
||||
|
@ -192,7 +192,7 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
@Override
|
||||
public Sequence<Result<TimeBoundaryResultValue>> mergeSequences(Sequence<Sequence<Result<TimeBoundaryResultValue>>> seqOfSequences)
|
||||
{
|
||||
return new MergeSequence<Result<TimeBoundaryResultValue>>(getOrdering(), seqOfSequences);
|
||||
return new MergeSequence<>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import io.druid.query.QueryWatcher;
|
|||
import io.druid.query.Result;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
@ -61,7 +62,7 @@ public class TimeBoundaryQueryRunnerFactory
|
|||
ExecutorService queryExecutor, Iterable<QueryRunner<Result<TimeBoundaryResultValue>>> queryRunners
|
||||
)
|
||||
{
|
||||
return new ChainedExecutionQueryRunner<Result<TimeBoundaryResultValue>>(
|
||||
return new ChainedExecutionQueryRunner<>(
|
||||
queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners
|
||||
);
|
||||
}
|
||||
|
@ -90,7 +91,7 @@ public class TimeBoundaryQueryRunnerFactory
|
|||
|
||||
final TimeBoundaryQuery legacyQuery = (TimeBoundaryQuery) input;
|
||||
|
||||
return new BaseSequence<Result<TimeBoundaryResultValue>, Iterator<Result<TimeBoundaryResultValue>>>(
|
||||
return new BaseSequence<>(
|
||||
new BaseSequence.IteratorMaker<Result<TimeBoundaryResultValue>, Iterator<Result<TimeBoundaryResultValue>>>()
|
||||
{
|
||||
@Override
|
||||
|
@ -102,10 +103,18 @@ public class TimeBoundaryQueryRunnerFactory
|
|||
);
|
||||
}
|
||||
|
||||
final DateTime minTime = legacyQuery.getBound().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)
|
||||
? null
|
||||
: adapter.getMinTime();
|
||||
final DateTime maxTime = legacyQuery.getBound().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)
|
||||
? null
|
||||
: adapter.getMaxTime();
|
||||
|
||||
|
||||
return legacyQuery.buildResult(
|
||||
adapter.getInterval().getStart(),
|
||||
adapter.getMinTime(),
|
||||
adapter.getMaxTime()
|
||||
minTime,
|
||||
maxTime
|
||||
).iterator();
|
||||
}
|
||||
|
||||
|
|
|
@ -99,6 +99,10 @@ public class TimeBoundaryResultValue
|
|||
|
||||
private DateTime getDateTimeValue(Object val)
|
||||
{
|
||||
if (val == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (val instanceof DateTime) {
|
||||
return (DateTime) val;
|
||||
} else if (val instanceof String) {
|
||||
|
|
|
@ -26,7 +26,6 @@ import io.druid.query.QueryRunnerHelper;
|
|||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.filter.Filters;
|
||||
|
@ -46,45 +45,43 @@ public class TimeseriesQueryEngine
|
|||
}
|
||||
|
||||
return QueryRunnerHelper.makeCursorBasedQuery(
|
||||
adapter,
|
||||
query.getQuerySegmentSpec().getIntervals(),
|
||||
Filters.convertDimensionFilters(query.getDimensionsFilter()),
|
||||
query.getGranularity(),
|
||||
new Function<Cursor, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
private final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
|
||||
private final List<PostAggregator> postAggregatorSpecs = query.getPostAggregatorSpecs();
|
||||
adapter,
|
||||
query.getQuerySegmentSpec().getIntervals(),
|
||||
Filters.convertDimensionFilters(query.getDimensionsFilter()),
|
||||
query.getGranularity(),
|
||||
new Function<Cursor, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
private final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
|
||||
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(Cursor cursor)
|
||||
{
|
||||
Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs);
|
||||
|
||||
try {
|
||||
while (!cursor.isDone()) {
|
||||
for (Aggregator aggregator : aggregators) {
|
||||
aggregator.aggregate();
|
||||
}
|
||||
cursor.advance();
|
||||
}
|
||||
|
||||
TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
|
||||
|
||||
for (Aggregator aggregator : aggregators) {
|
||||
bob.addMetric(aggregator);
|
||||
}
|
||||
|
||||
Result<TimeseriesResultValue> retVal = bob.build();
|
||||
return retVal;
|
||||
}
|
||||
finally {
|
||||
// cleanup
|
||||
for (Aggregator agg : aggregators) {
|
||||
agg.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(Cursor cursor)
|
||||
{
|
||||
Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs);
|
||||
try {
|
||||
while (!cursor.isDone()) {
|
||||
for (Aggregator aggregator : aggregators) {
|
||||
aggregator.aggregate();
|
||||
}
|
||||
cursor.advance();
|
||||
}
|
||||
|
||||
TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
|
||||
|
||||
for (Aggregator aggregator : aggregators) {
|
||||
bob.addMetric(aggregator);
|
||||
}
|
||||
|
||||
Result<TimeseriesResultValue> retVal = bob.build();
|
||||
return retVal;
|
||||
}
|
||||
finally {
|
||||
// cleanup
|
||||
for (Aggregator agg : aggregators) {
|
||||
agg.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package io.druid.query.topn;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
|
@ -233,7 +233,7 @@ public class PooledTopNAlgorithm
|
|||
if (resultsBufHolder != null) {
|
||||
resultsBufHolder.get().clear();
|
||||
}
|
||||
Closeables.closeQuietly(resultsBufHolder);
|
||||
CloseQuietly.close(resultsBufHolder);
|
||||
}
|
||||
|
||||
public static class PooledTopNParams extends TopNParams
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||
|
@ -95,7 +95,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
return column.length();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.segment;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -31,6 +32,9 @@ import com.google.common.io.ByteStreams;
|
|||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Module;
|
||||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
|
@ -41,9 +45,13 @@ import com.metamx.common.io.smoosh.SmooshedWriter;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.common.utils.SerializerUtils;
|
||||
import io.druid.guice.ConfigProvider;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.DruidProcessingConfig;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ColumnBuilder;
|
||||
import io.druid.segment.column.ColumnConfig;
|
||||
import io.druid.segment.column.ColumnDescriptor;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.ArrayIndexed;
|
||||
|
@ -90,6 +98,9 @@ public class IndexIO
|
|||
{
|
||||
public static final byte V8_VERSION = 0x8;
|
||||
public static final byte V9_VERSION = 0x9;
|
||||
public static final int CURRENT_VERSION_ID = V9_VERSION;
|
||||
|
||||
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
|
||||
|
||||
private static final Map<Integer, IndexLoader> indexLoaders =
|
||||
ImmutableMap.<Integer, IndexLoader>builder()
|
||||
|
@ -107,13 +118,33 @@ public class IndexIO
|
|||
|
||||
private static final EmittingLogger log = new EmittingLogger(IndexIO.class);
|
||||
private static final SerializerUtils serializerUtils = new SerializerUtils();
|
||||
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
|
||||
|
||||
// This should really be provided by DI, should be changed once we switch around to using a DI framework
|
||||
private static final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
private static final ObjectMapper mapper;
|
||||
protected static final ColumnConfig columnConfig;
|
||||
|
||||
static {
|
||||
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(
|
||||
ImmutableList.<Module>of(
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
ConfigProvider.bind(
|
||||
binder,
|
||||
DruidProcessingConfig.class,
|
||||
ImmutableMap.of("base_path", "druid.processing")
|
||||
);
|
||||
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
mapper = injector.getInstance(ObjectMapper.class);
|
||||
columnConfig = injector.getInstance(ColumnConfig.class);
|
||||
}
|
||||
|
||||
private static volatile IndexIOHandler handler = null;
|
||||
public static final int CURRENT_VERSION_ID = V9_VERSION;
|
||||
|
||||
@Deprecated
|
||||
public static MMappedIndex mapDir(final File inDir) throws IOException
|
||||
|
@ -165,15 +196,10 @@ public class IndexIO
|
|||
}
|
||||
|
||||
final File indexFile = new File(inDir, "index.drd");
|
||||
InputStream in = null;
|
||||
int version;
|
||||
try {
|
||||
in = new FileInputStream(indexFile);
|
||||
try (InputStream in = new FileInputStream(indexFile)) {
|
||||
version = in.read();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
}
|
||||
return version;
|
||||
}
|
||||
|
||||
|
@ -194,8 +220,8 @@ public class IndexIO
|
|||
case 2:
|
||||
case 3:
|
||||
log.makeAlert("Attempt to load segment of version <= 3.")
|
||||
.addData("version", version)
|
||||
.emit();
|
||||
.addData("version", version)
|
||||
.emit();
|
||||
return false;
|
||||
case 4:
|
||||
case 5:
|
||||
|
@ -631,7 +657,10 @@ public class IndexIO
|
|||
.setHasMultipleValues(true)
|
||||
.setDictionaryEncodedColumn(
|
||||
new DictionaryEncodedColumnSupplier(
|
||||
index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension))
|
||||
index.getDimValueLookup(dimension),
|
||||
null,
|
||||
index.getDimColumn(dimension),
|
||||
columnConfig.columnCacheSizeBytes()
|
||||
)
|
||||
)
|
||||
.setBitmapIndex(
|
||||
|
@ -743,7 +772,7 @@ public class IndexIO
|
|||
ColumnDescriptor serde = mapper.readValue(
|
||||
serializerUtils.readString(byteBuffer), ColumnDescriptor.class
|
||||
);
|
||||
return serde.read(byteBuffer);
|
||||
return serde.read(byteBuffer, columnConfig);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -38,6 +38,7 @@ import com.metamx.collections.spatial.RTree;
|
|||
import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.guava.MergeIterable;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
|
@ -50,6 +51,7 @@ 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.ColumnConfig;
|
||||
import io.druid.segment.data.ByteBufferWriter;
|
||||
import io.druid.segment.data.CompressedLongsSupplierSerializer;
|
||||
import io.druid.segment.data.ConciseCompressedIndexedInts;
|
||||
|
@ -438,9 +440,9 @@ public class IndexMerger
|
|||
serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime));
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(channel);
|
||||
CloseQuietly.close(channel);
|
||||
channel = null;
|
||||
Closeables.closeQuietly(fileOutputStream);
|
||||
CloseQuietly.close(fileOutputStream);
|
||||
fileOutputStream = null;
|
||||
}
|
||||
IndexIO.checkFileSize(indexFile);
|
||||
|
@ -881,7 +883,7 @@ public class IndexMerger
|
|||
);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(channel);
|
||||
CloseQuietly.close(channel);
|
||||
channel = null;
|
||||
}
|
||||
IndexIO.checkFileSize(indexFile);
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.segment;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.segment.data.ConciseCompressedIndexedInts;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedFloats;
|
||||
|
@ -118,9 +118,9 @@ public class MMappedIndexAdapter implements IndexableAdapter
|
|||
{
|
||||
final boolean hasNext = currRow < numRows;
|
||||
if (!hasNext && !done) {
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
for (IndexedFloats floatMetric : floatMetrics) {
|
||||
Closeables.closeQuietly(floatMetric);
|
||||
CloseQuietly.close(floatMetric);
|
||||
}
|
||||
done = true;
|
||||
}
|
||||
|
|
|
@ -20,11 +20,11 @@
|
|||
package io.druid.segment;
|
||||
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.google.common.io.OutputSupplier;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.common.utils.SerializerUtils;
|
||||
import io.druid.segment.data.CompressedFloatsIndexedSupplier;
|
||||
import io.druid.segment.data.CompressedFloatsSupplierSerializer;
|
||||
|
@ -84,8 +84,8 @@ public class MetricHolder
|
|||
ByteStreams.copy(in, out);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(out);
|
||||
Closeables.closeQuietly(in);
|
||||
CloseQuietly.close(out);
|
||||
CloseQuietly.close(in);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,8 +22,8 @@ package io.druid.segment;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
|
@ -208,10 +208,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
{
|
||||
final boolean hasNext = currRow < numRows;
|
||||
if (!hasNext && !done) {
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
for (Object metric : metrics) {
|
||||
if (metric instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) metric);
|
||||
CloseQuietly.close((Closeable) metric);
|
||||
}
|
||||
}
|
||||
done = true;
|
||||
|
|
|
@ -21,9 +21,10 @@ package io.druid.segment;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
|
@ -38,12 +39,12 @@ import io.druid.segment.column.ValueType;
|
|||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.data.Offset;
|
||||
import io.druid.segment.data.SingleIndexedInts;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -109,7 +110,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return new DateTime(column.getLongSingleValueRow(0));
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -122,7 +123,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return new DateTime(column.getLongSingleValueRow(column.length() - 1));
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -186,6 +187,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
{
|
||||
final Offset baseOffset = offset.clone();
|
||||
|
||||
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
|
||||
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
|
||||
final Map<String, ComplexColumn> complexColumnCache = Maps.newHashMap();
|
||||
final Map<String, Object> objectColumnCache = Maps.newHashMap();
|
||||
|
@ -270,12 +272,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
public DimensionSelector makeDimensionSelector(String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
|
||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName);
|
||||
final Column columnDesc = index.getColumn(dimensionName);
|
||||
if (columnDesc == null) {
|
||||
return null;
|
||||
|
||||
if (cachedColumn == null && columnDesc != null) {
|
||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||
dictionaryColumnCache.put(dimensionName, cachedColumn);
|
||||
}
|
||||
|
||||
final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding();
|
||||
final DictionaryEncodedColumn column = cachedColumn;
|
||||
|
||||
if (column == null) {
|
||||
return null;
|
||||
|
@ -313,7 +319,27 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return new SingleIndexedInts(column.getSingleValueRow(cursorOffset.getOffset()));
|
||||
// using an anonymous class is faster than creating a class that stores a copy of the value
|
||||
return new IndexedInts()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return column.getSingleValueRow(cursorOffset.getOffset());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset()));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -535,16 +561,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (GenericColumn column : genericColumnCache.values()) {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (ComplexColumn complexColumn : complexColumnCache.values()) {
|
||||
Closeables.closeQuietly(complexColumn);
|
||||
CloseQuietly.close(complexColumn);
|
||||
}
|
||||
for (Object column : objectColumnCache.values()) {
|
||||
if(column instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) column);
|
||||
CloseQuietly.close((Closeable) column);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -620,6 +649,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
*/
|
||||
public Sequence<Cursor> build()
|
||||
{
|
||||
final Map<String, DictionaryEncodedColumn> dictionaryColumnCache = Maps.newHashMap();
|
||||
final Map<String, GenericColumn> genericColumnCache = Maps.newHashMap();
|
||||
final Map<String, ComplexColumn> complexColumnCache = Maps.newHashMap();
|
||||
final Map<String, Object> objectColumnCache = Maps.newHashMap();
|
||||
|
@ -697,41 +727,45 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
public DimensionSelector makeDimensionSelector(String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
final Column column = index.getColumn(dimensionName);
|
||||
if (column == null) {
|
||||
return null;
|
||||
|
||||
DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName);
|
||||
final Column columnDesc = index.getColumn(dimensionName);
|
||||
|
||||
if (cachedColumn == null && columnDesc != null) {
|
||||
cachedColumn = columnDesc.getDictionaryEncoding();
|
||||
dictionaryColumnCache.put(dimensionName, cachedColumn);
|
||||
}
|
||||
|
||||
final DictionaryEncodedColumn dict = column.getDictionaryEncoding();
|
||||
final DictionaryEncodedColumn column = cachedColumn;
|
||||
|
||||
if (dict == null) {
|
||||
if (column == null) {
|
||||
return null;
|
||||
} else if (column.getCapabilities().hasMultipleValues()) {
|
||||
} else if (columnDesc.getCapabilities().hasMultipleValues()) {
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return dict.getMultiValueRow(currRow);
|
||||
return column.getMultiValueRow(currRow);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return dict.getCardinality();
|
||||
return column.getCardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
final String retVal = dict.lookupName(id);
|
||||
final String retVal = column.lookupName(id);
|
||||
return retVal == null ? "" : retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return dict.lookupId(name);
|
||||
return column.lookupId(name);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
|
@ -740,25 +774,45 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return new SingleIndexedInts(dict.getSingleValueRow(currRow));
|
||||
// using an anonymous class is faster than creating a class that stores a copy of the value
|
||||
return new IndexedInts()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return column.getSingleValueRow(currRow);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return Iterators.singletonIterator(column.getSingleValueRow(currRow));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return dict.getCardinality();
|
||||
return column.getCardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return dict.lookupName(id);
|
||||
return column.lookupName(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return dict.lookupId(name);
|
||||
return column.lookupId(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -962,16 +1016,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) {
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (GenericColumn column : genericColumnCache.values()) {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
for (ComplexColumn complexColumn : complexColumnCache.values()) {
|
||||
Closeables.closeQuietly(complexColumn);
|
||||
CloseQuietly.close(complexColumn);
|
||||
}
|
||||
for (Object column : objectColumnCache.values()) {
|
||||
if (column instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) column);
|
||||
CloseQuietly.close((Closeable) column);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -979,31 +1036,4 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static class NullDimensionSelector implements DimensionSelector
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return new SingleIndexedInts(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* 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.segment.column;
|
||||
|
||||
public interface ColumnConfig
|
||||
{
|
||||
public int columnCacheSizeBytes();
|
||||
}
|
|
@ -92,14 +92,14 @@ public class ColumnDescriptor
|
|||
}
|
||||
}
|
||||
|
||||
public Column read(ByteBuffer buffer)
|
||||
public Column read(ByteBuffer buffer, ColumnConfig columnConfig)
|
||||
{
|
||||
final ColumnBuilder builder = new ColumnBuilder()
|
||||
.setType(valueType)
|
||||
.setHasMultipleValues(hasMultipleValues);
|
||||
|
||||
for (ColumnPartSerde part : parts) {
|
||||
part.read(buffer, builder);
|
||||
part.read(buffer, builder, columnConfig);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
|
|
|
@ -21,9 +21,11 @@ package io.druid.segment.column;
|
|||
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface DictionaryEncodedColumn
|
||||
public interface DictionaryEncodedColumn extends Closeable
|
||||
{
|
||||
public int length();
|
||||
public boolean hasMultipleValues();
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.segment.column;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -68,7 +68,7 @@ class SimpleColumn implements Column
|
|||
return column.length();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,8 @@ import io.druid.segment.data.IndexedInts;
|
|||
import io.druid.segment.data.VSizeIndexed;
|
||||
import io.druid.segment.data.VSizeIndexedInts;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
|
||||
|
@ -84,4 +86,10 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
|
|||
{
|
||||
return lookups.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
lookups.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
* 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
|
||||
|
@ -19,35 +19,8 @@
|
|||
|
||||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.collect.Iterators;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SingleIndexedInts implements IndexedInts
|
||||
{
|
||||
private final int value;
|
||||
|
||||
public SingleIndexedInts(int value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return Iterators.singletonIterator(value);
|
||||
}
|
||||
}
|
||||
* Implementing CacheableObjectStrategy instead of ObjectSrategy indicates
|
||||
* that a column scan may cache the results of fromByteBuffer
|
||||
*/
|
||||
public interface CacheableObjectStrategy<T> extends ObjectStrategy<T> {}
|
|
@ -25,6 +25,7 @@ import com.google.common.io.Closeables;
|
|||
import com.google.common.primitives.Floats;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidResourceHolder;
|
||||
|
||||
|
@ -123,7 +124,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
|
|||
|
||||
private void loadBuffer(int bufferNum)
|
||||
{
|
||||
Closeables.closeQuietly(holder);
|
||||
CloseQuietly.close(holder);
|
||||
holder = baseFloatBuffers.get(bufferNum);
|
||||
buffer = holder.get();
|
||||
currIndex = bufferNum;
|
||||
|
|
|
@ -106,17 +106,11 @@ public class CompressedFloatsSupplierSerializer
|
|||
|
||||
flattener.close();
|
||||
|
||||
OutputStream out = null;
|
||||
try {
|
||||
out = consolidatedOut.getOutput();
|
||||
|
||||
try (OutputStream out = consolidatedOut.getOutput()) {
|
||||
out.write(CompressedFloatsIndexedSupplier.version);
|
||||
out.write(Ints.toByteArray(numInserted));
|
||||
out.write(Ints.toByteArray(sizePer));
|
||||
ByteStreams.copy(flattener.combineStreams(), out);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.io.Closeables;
|
|||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidResourceHolder;
|
||||
|
||||
|
@ -122,7 +123,7 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
|||
|
||||
private void loadBuffer(int bufferNum)
|
||||
{
|
||||
Closeables.closeQuietly(holder);
|
||||
CloseQuietly.close(holder);
|
||||
holder = baseLongBuffers.get(bufferNum);
|
||||
buffer = holder.get();
|
||||
currIndex = bufferNum;
|
||||
|
|
|
@ -100,17 +100,11 @@ public class CompressedLongsSupplierSerializer
|
|||
|
||||
flattener.close();
|
||||
|
||||
OutputStream out = null;
|
||||
try {
|
||||
out = consolidatedOut.getOutput();
|
||||
|
||||
try (OutputStream out = consolidatedOut.getOutput()) {
|
||||
out.write(CompressedLongsIndexedSupplier.version);
|
||||
out.write(Ints.toByteArray(numInserted));
|
||||
out.write(Ints.toByteArray(sizePer));
|
||||
ByteStreams.copy(flattener.combineStreams(), out);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue