mirror of https://github.com/apache/druid.git
fixed merge conflicts
This commit is contained in:
commit
c4397bd65a
|
@ -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
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,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;
|
||||
|
@ -65,8 +66,6 @@ public class HadoopIndexTask extends AbstractTask
|
|||
extensionsConfig = Initialization.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 +114,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 +162,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 +182,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
|
||||
);
|
||||
|
|
|
@ -24,9 +24,9 @@ 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;
|
||||
|
@ -44,8 +44,8 @@ import io.druid.query.QueryRunnerFactory;
|
|||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import io.druid.query.QueryToolChest;
|
||||
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 +353,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 +375,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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -135,7 +135,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")
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -121,7 +121,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...
|
||||
|
|
|
@ -132,17 +132,17 @@ public class DruidDefaultSerializersModule extends SimpleModule
|
|||
public void serialize(Yielder yielder, final JsonGenerator jgen, SerializerProvider provider)
|
||||
throws IOException, JsonProcessingException
|
||||
{
|
||||
jgen.writeStartArray();
|
||||
try {
|
||||
jgen.writeStartArray();
|
||||
while (!yielder.isDone()) {
|
||||
final Object o = yielder.get();
|
||||
jgen.writeObject(o);
|
||||
yielder = yielder.next(null);
|
||||
}
|
||||
jgen.writeEndArray();
|
||||
} finally {
|
||||
yielder.close();
|
||||
}
|
||||
jgen.writeEndArray();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.guava.Yielders;
|
|||
import com.metamx.common.guava.YieldingAccumulator;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -55,16 +56,11 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
|
|||
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||
{
|
||||
if (query.getContextBySegment(false)) {
|
||||
|
||||
final Sequence<T> baseSequence = base.run(query, context);
|
||||
return new Sequence<T>()
|
||||
{
|
||||
@Override
|
||||
public <OutType> OutType accumulate(OutType initValue, Accumulator<OutType, T> accumulator)
|
||||
{
|
||||
List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
|
||||
|
||||
return accumulator.accumulate(
|
||||
initValue,
|
||||
final List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
|
||||
return Sequences.simple(
|
||||
Arrays.asList(
|
||||
(T) new Result<BySegmentResultValueClass<T>>(
|
||||
timestamp,
|
||||
new BySegmentResultValueClass<T>(
|
||||
|
@ -73,29 +69,8 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
|
|||
query.getIntervals().get(0)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
|
||||
{
|
||||
List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
|
||||
|
||||
final OutType retVal = accumulator.accumulate(
|
||||
initValue,
|
||||
(T) new Result<BySegmentResultValueClass<T>>(
|
||||
timestamp,
|
||||
new BySegmentResultValueClass<T>(
|
||||
results,
|
||||
segmentIdentifier,
|
||||
query.getIntervals().get(0)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
return Yielders.done(retVal, null);
|
||||
}
|
||||
};
|
||||
)
|
||||
);
|
||||
}
|
||||
return base.run(query, context);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -89,7 +89,6 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
|
|||
@Override
|
||||
public Sequence<Row> run(final Query<Row> queryParam, final Map<String, Object> context)
|
||||
{
|
||||
|
||||
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;
|
||||
|
@ -54,7 +54,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)
|
||||
{
|
||||
|
|
|
@ -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,6 +45,7 @@ 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;
|
||||
|
@ -60,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
|
||||
|
||||
|
@ -82,7 +87,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
@Override
|
||||
public Sequence<Row> run(Query<Row> input, Map<String, Object> context)
|
||||
{
|
||||
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, context);
|
||||
} else {
|
||||
return runner.run(input, context);
|
||||
|
@ -93,33 +98,45 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
|
||||
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, Object> context)
|
||||
{
|
||||
|
||||
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, context);
|
||||
IncrementalIndexStorageAdapter adapter
|
||||
= new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult));
|
||||
result = engine.process(query, adapter);
|
||||
final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, context);
|
||||
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, context);
|
||||
return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query, context))));
|
||||
}
|
||||
|
||||
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>()
|
||||
{
|
||||
|
@ -129,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)
|
||||
|
@ -153,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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -68,8 +68,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(
|
||||
|
@ -79,8 +79,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()));
|
||||
}
|
||||
}
|
||||
)
|
||||
|
@ -112,7 +112,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
|
||||
|
@ -147,9 +147,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
|
||||
|
@ -178,11 +178,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))
|
||||
);
|
||||
|
@ -193,7 +193,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.List;
|
||||
|
@ -63,7 +64,7 @@ public class TimeBoundaryQueryRunnerFactory
|
|||
ExecutorService queryExecutor, Iterable<QueryRunner<Result<TimeBoundaryResultValue>>> queryRunners
|
||||
)
|
||||
{
|
||||
return new ChainedExecutionQueryRunner<Result<TimeBoundaryResultValue>>(
|
||||
return new ChainedExecutionQueryRunner<>(
|
||||
queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners
|
||||
);
|
||||
}
|
||||
|
@ -95,7 +96,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
|
||||
|
@ -107,10 +108,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) {
|
||||
|
|
|
@ -25,7 +25,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.SegmentMissingException;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -165,15 +165,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 +189,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:
|
||||
|
|
|
@ -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;
|
||||
|
@ -438,9 +439,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 +882,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;
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Predicates;
|
||||
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;
|
||||
|
@ -109,7 +109,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return new DateTime(column.getLongSingleValueRow(0));
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -122,7 +122,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
return new DateTime(column.getLongSingleValueRow(column.length() - 1));
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
CloseQuietly.close(column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -535,16 +535,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -962,16 +962,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(timestamps);
|
||||
CloseQuietly.close(timestamps);
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.ning.compress.lzf.ChunkEncoder;
|
||||
import com.ning.compress.lzf.LZFChunk;
|
||||
import com.ning.compress.lzf.LZFDecoder;
|
||||
|
@ -74,7 +74,7 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
|
|||
buf.put(outputBytes, 0, numDecompressedBytes);
|
||||
buf.flip();
|
||||
|
||||
Closeables.closeQuietly(outputBytesHolder);
|
||||
CloseQuietly.close(outputBytesHolder);
|
||||
|
||||
return new ResourceHolder<T>()
|
||||
{
|
||||
|
@ -105,7 +105,7 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
|
|||
|
||||
final ResourceHolder<ChunkEncoder> encoder = CompressedPools.getChunkEncoder();
|
||||
LZFChunk chunk = encoder.get().encodeChunk(buf.array(), 0, buf.array().length);
|
||||
Closeables.closeQuietly(encoder);
|
||||
CloseQuietly.close(encoder);
|
||||
|
||||
return chunk.getData();
|
||||
}
|
||||
|
|
|
@ -21,9 +21,9 @@ package io.druid.segment.data;
|
|||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.Closeable;
|
||||
|
@ -73,14 +73,14 @@ public class GenericIndexed<T> implements Indexed<T>
|
|||
allowReverseLookup = false;
|
||||
}
|
||||
if (prevVal instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) prevVal);
|
||||
CloseQuietly.close((Closeable) prevVal);
|
||||
}
|
||||
|
||||
prevVal = next;
|
||||
++count;
|
||||
}
|
||||
if (prevVal instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) prevVal);
|
||||
CloseQuietly.close((Closeable) prevVal);
|
||||
}
|
||||
|
||||
ByteArrayOutputStream headerBytes = new ByteArrayOutputStream(4 + (count * 4));
|
||||
|
@ -98,7 +98,7 @@ public class GenericIndexed<T> implements Indexed<T>
|
|||
valueBytes.write(bytes);
|
||||
|
||||
if (object instanceof Closeable) {
|
||||
Closeables.closeQuietly((Closeable) object);
|
||||
CloseQuietly.close((Closeable) object);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.segment.data;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidResourceHolder;
|
||||
|
||||
|
@ -153,7 +153,7 @@ public class InMemoryCompressedFloats implements IndexedFloats
|
|||
private void loadBuffer(int bufferNum)
|
||||
{
|
||||
loadBuffer = null;
|
||||
Closeables.closeQuietly(holder);
|
||||
CloseQuietly.close(holder);
|
||||
final byte[] compressedBytes = compressedBuffers.get(bufferNum);
|
||||
holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length);
|
||||
loadBuffer = holder.get();
|
||||
|
@ -191,6 +191,6 @@ public class InMemoryCompressedFloats implements IndexedFloats
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
Closeables.closeQuietly(holder);
|
||||
CloseQuietly.close(holder);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.collections.ResourceHolder;
|
||||
import io.druid.collections.StupidResourceHolder;
|
||||
|
||||
|
@ -163,7 +164,7 @@ public class InMemoryCompressedLongs implements IndexedLongs
|
|||
private void loadBuffer(int bufferNum)
|
||||
{
|
||||
loadBuffer = null;
|
||||
Closeables.closeQuietly(holder);
|
||||
CloseQuietly.close(holder);
|
||||
final byte[] compressedBytes = compressedBuffers.get(bufferNum);
|
||||
holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length);
|
||||
loadBuffer = holder.get();
|
||||
|
|
|
@ -46,6 +46,7 @@ import io.druid.segment.DimensionSelector;
|
|||
import io.druid.segment.FloatColumnSelector;
|
||||
import io.druid.segment.ObjectColumnSelector;
|
||||
import io.druid.segment.TimestampColumnSelector;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||
import io.druid.segment.serde.ComplexMetricSerde;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
|
@ -53,6 +54,7 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
|
@ -133,17 +135,29 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
);
|
||||
}
|
||||
|
||||
public int add(InputRow row)
|
||||
{
|
||||
// this is an ugly workaround to call ComplexMetricExtractor.extractValue at ingestion time
|
||||
return add(row, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new row. The row might correspond with another row that already exists, in which case this will
|
||||
* update that row instead of inserting a new one.
|
||||
* <p/>
|
||||
* This is *not* thread-safe. Calls to add() should always happen on the same thread.
|
||||
*
|
||||
* Calls to add() are thread safe.
|
||||
*
|
||||
* Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that
|
||||
* should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics.
|
||||
*
|
||||
* @param row the row of data to add
|
||||
* @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input
|
||||
* value for aggregators that return metrics other than float.
|
||||
*
|
||||
* @return the number of rows in the data set after adding the InputRow
|
||||
*/
|
||||
public int add(InputRow row)
|
||||
public int add(InputRow row, final boolean deserializeComplexMetrics)
|
||||
{
|
||||
row = spatialDimensionRowFormatter.formatRow(row);
|
||||
|
||||
|
@ -186,7 +200,7 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
dims = newDims;
|
||||
}
|
||||
|
||||
TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims);
|
||||
final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims);
|
||||
|
||||
Aggregator[] aggs = facts.get(key);
|
||||
if (aggs == null) {
|
||||
|
@ -231,54 +245,108 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
final String typeName = agg.getTypeName();
|
||||
final String columnName = column.toLowerCase();
|
||||
|
||||
if (typeName.equals("float")) {
|
||||
return new ObjectColumnSelector<Float>()
|
||||
final ObjectColumnSelector<Object> rawColumnSelector = new ObjectColumnSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return Float.TYPE;
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Float get()
|
||||
public Object get()
|
||||
{
|
||||
return in.getFloatMetric(columnName);
|
||||
return in.getRaw(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
if(!deserializeComplexMetrics) {
|
||||
return rawColumnSelector;
|
||||
} else {
|
||||
if (typeName.equals("float")) {
|
||||
return rawColumnSelector;
|
||||
}
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
if (serde == null) {
|
||||
throw new ISE("Don't know how to handle type[%s]", typeName);
|
||||
}
|
||||
|
||||
final ComplexMetricExtractor extractor = serde.getExtractor();
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return extractor.extractedClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return extractor.extractValue(in, columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
|
||||
if (serde == null) {
|
||||
throw new ISE("Don't know how to handle type[%s]", typeName);
|
||||
}
|
||||
|
||||
final ComplexMetricExtractor extractor = serde.getExtractor();
|
||||
|
||||
return new ObjectColumnSelector()
|
||||
{
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return extractor.extractedClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get()
|
||||
{
|
||||
return extractor.extractValue(in, columnName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(String dimension)
|
||||
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||
{
|
||||
// we should implement this, but this is going to be rewritten soon anyways
|
||||
throw new UnsupportedOperationException(
|
||||
"Incremental index aggregation does not support dimension selectors"
|
||||
);
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
final List<String> dimensionValues = in.getDimension(dimensionName);
|
||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
||||
if (dimensionValues != null) {
|
||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
||||
vals.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new IndexedInts()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return vals.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(int index)
|
||||
{
|
||||
return vals.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Integer> iterator()
|
||||
{
|
||||
return vals.iterator();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
throw new UnsupportedOperationException("value cardinality is unknown in incremental index");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return in.getDimension(dimensionName).get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return in.getDimension(dimensionName).indexOf(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,12 +23,14 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Floats;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.parsers.ParseException;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.SpatialDimensionSchema;
|
||||
|
||||
|
@ -134,14 +136,20 @@ public class SpatialDimensionRowFormatter
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension) {
|
||||
public Object getRaw(String dimension)
|
||||
{
|
||||
return row.getRaw(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloatMetric(String metric)
|
||||
{
|
||||
return row.getFloatMetric(metric);
|
||||
try {
|
||||
return row.getFloatMetric(metric);
|
||||
}
|
||||
catch (ParseException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -67,6 +67,7 @@ public class ChainedExecutionQueryRunnerTest
|
|||
);
|
||||
|
||||
final CountDownLatch queriesStarted = new CountDownLatch(2);
|
||||
final CountDownLatch queriesInterrupted = new CountDownLatch(2);
|
||||
final CountDownLatch queryIsRegistered = new CountDownLatch(1);
|
||||
|
||||
Capture<ListenableFuture> capturedFuture = new Capture<>();
|
||||
|
@ -88,9 +89,9 @@ public class ChainedExecutionQueryRunnerTest
|
|||
|
||||
EasyMock.replay(watcher);
|
||||
|
||||
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>(
|
||||
exec,
|
||||
Ordering.<Integer>natural(),
|
||||
|
@ -138,11 +139,14 @@ public class ChainedExecutionQueryRunnerTest
|
|||
Assert.assertTrue(e.getCause() instanceof QueryInterruptedException);
|
||||
cause = (QueryInterruptedException)e.getCause();
|
||||
}
|
||||
Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS));
|
||||
Assert.assertNotNull(cause);
|
||||
Assert.assertTrue(future.isCancelled());
|
||||
Assert.assertTrue(runner1.hasStarted);
|
||||
Assert.assertTrue(runner2.hasStarted);
|
||||
Assert.assertFalse(runner3.hasStarted);
|
||||
Assert.assertTrue(runner1.interrupted);
|
||||
Assert.assertTrue(runner2.interrupted);
|
||||
Assert.assertTrue(!runner3.hasStarted || runner3.interrupted);
|
||||
Assert.assertFalse(runner1.hasCompleted);
|
||||
Assert.assertFalse(runner2.hasCompleted);
|
||||
Assert.assertFalse(runner3.hasCompleted);
|
||||
|
@ -171,6 +175,7 @@ public class ChainedExecutionQueryRunnerTest
|
|||
);
|
||||
|
||||
final CountDownLatch queriesStarted = new CountDownLatch(2);
|
||||
final CountDownLatch queriesInterrupted = new CountDownLatch(2);
|
||||
final CountDownLatch queryIsRegistered = new CountDownLatch(1);
|
||||
|
||||
Capture<ListenableFuture> capturedFuture = new Capture<>();
|
||||
|
@ -192,9 +197,9 @@ public class ChainedExecutionQueryRunnerTest
|
|||
|
||||
EasyMock.replay(watcher);
|
||||
|
||||
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted);
|
||||
DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted);
|
||||
ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>(
|
||||
exec,
|
||||
Ordering.<Integer>natural(),
|
||||
|
@ -211,7 +216,7 @@ public class ChainedExecutionQueryRunnerTest
|
|||
.dataSource("test")
|
||||
.intervals("2014/2015")
|
||||
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
|
||||
.context(ImmutableMap.<String, Object>of("timeout", (100), "queryId", "test"))
|
||||
.context(ImmutableMap.<String, Object>of("timeout", 100, "queryId", "test"))
|
||||
.build(),
|
||||
context
|
||||
);
|
||||
|
@ -231,10 +236,10 @@ public class ChainedExecutionQueryRunnerTest
|
|||
Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS));
|
||||
Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS));
|
||||
|
||||
// cancel the query
|
||||
Assert.assertTrue(capturedFuture.hasCaptured());
|
||||
ListenableFuture future = capturedFuture.getValue();
|
||||
|
||||
// wait for query to time out
|
||||
QueryInterruptedException cause = null;
|
||||
try {
|
||||
resultFuture.get();
|
||||
|
@ -243,11 +248,14 @@ public class ChainedExecutionQueryRunnerTest
|
|||
Assert.assertEquals("Query timeout", e.getCause().getMessage());
|
||||
cause = (QueryInterruptedException)e.getCause();
|
||||
}
|
||||
Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS));
|
||||
Assert.assertNotNull(cause);
|
||||
Assert.assertTrue(future.isCancelled());
|
||||
Assert.assertTrue(runner1.hasStarted);
|
||||
Assert.assertTrue(runner2.hasStarted);
|
||||
Assert.assertFalse(runner3.hasStarted);
|
||||
Assert.assertTrue(runner1.interrupted);
|
||||
Assert.assertTrue(runner2.interrupted);
|
||||
Assert.assertTrue(!runner3.hasStarted || runner3.interrupted);
|
||||
Assert.assertFalse(runner1.hasCompleted);
|
||||
Assert.assertFalse(runner2.hasCompleted);
|
||||
Assert.assertFalse(runner3.hasCompleted);
|
||||
|
@ -257,21 +265,27 @@ public class ChainedExecutionQueryRunnerTest
|
|||
|
||||
private static class DyingQueryRunner implements QueryRunner<Integer>
|
||||
{
|
||||
private final CountDownLatch latch;
|
||||
private final CountDownLatch start;
|
||||
private final CountDownLatch stop;
|
||||
|
||||
private boolean hasStarted = false;
|
||||
private boolean hasCompleted = false;
|
||||
private boolean interrupted = false;
|
||||
|
||||
public DyingQueryRunner(CountDownLatch latch)
|
||||
public DyingQueryRunner(CountDownLatch start, CountDownLatch stop)
|
||||
{
|
||||
this.latch = latch;
|
||||
this.start = start;
|
||||
this.stop = stop;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<Integer> run(Query<Integer> query, Map<String, Object> context)
|
||||
{
|
||||
hasStarted = true;
|
||||
latch.countDown();
|
||||
start.countDown();
|
||||
if (Thread.interrupted()) {
|
||||
interrupted = true;
|
||||
stop.countDown();
|
||||
throw new QueryInterruptedException("I got killed");
|
||||
}
|
||||
|
||||
|
@ -280,10 +294,13 @@ public class ChainedExecutionQueryRunnerTest
|
|||
Thread.sleep(500);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
interrupted = true;
|
||||
stop.countDown();
|
||||
throw new QueryInterruptedException("I got killed");
|
||||
}
|
||||
|
||||
hasCompleted = true;
|
||||
stop.countDown();
|
||||
return Sequences.simple(Lists.newArrayList(123));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import io.druid.query.aggregation.CountAggregatorFactory;
|
|||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
|
@ -110,6 +111,11 @@ public class QueryRunnerTestHelper
|
|||
"uniques",
|
||||
"quality_uniques"
|
||||
);
|
||||
public static final CardinalityAggregatorFactory qualityCardinality = new CardinalityAggregatorFactory(
|
||||
"cardinality",
|
||||
Arrays.asList("quality"),
|
||||
false
|
||||
);
|
||||
public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null);
|
||||
public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows");
|
||||
public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index");
|
||||
|
|
|
@ -42,8 +42,13 @@ import io.druid.query.QueryRunnerTestHelper;
|
|||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.MaxAggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.dimension.ExtractionDimensionSpec;
|
||||
|
@ -52,6 +57,7 @@ import io.druid.query.filter.JavaScriptDimFilter;
|
|||
import io.druid.query.filter.RegexDimFilter;
|
||||
import io.druid.query.groupby.having.EqualToHavingSpec;
|
||||
import io.druid.query.groupby.having.GreaterThanHavingSpec;
|
||||
import io.druid.query.groupby.having.HavingSpec;
|
||||
import io.druid.query.groupby.having.OrHavingSpec;
|
||||
import io.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
import io.druid.query.groupby.orderby.LimitSpec;
|
||||
|
@ -212,6 +218,36 @@ public class GroupByQueryRunnerTest
|
|||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithCardinality()
|
||||
{
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
QueryRunnerTestHelper.qualityCardinality
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow(
|
||||
"2011-04-01",
|
||||
"rows",
|
||||
26L,
|
||||
"cardinality",
|
||||
QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Row> results = runQuery(query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithDimExtractionFn()
|
||||
{
|
||||
|
@ -1037,6 +1073,278 @@ public class GroupByQueryRunnerTest
|
|||
Assert.assertFalse(results.iterator().hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubqueryWithPostAggregators()
|
||||
{
|
||||
final GroupByQuery subquery = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx_subagg", "index")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx_subpostagg", "+", Arrays.<PostAggregator>asList(
|
||||
new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"),
|
||||
new ConstantPostAggregator("thousand", 1000, 1000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
final GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(subquery)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("alias", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
new LongSumAggregatorFactory("rows", "rows"),
|
||||
new LongSumAggregatorFactory("idx", "idx_subpostagg")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000, 10000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0),
|
||||
createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0),
|
||||
createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0),
|
||||
createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0),
|
||||
createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0),
|
||||
createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0),
|
||||
createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0),
|
||||
createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0),
|
||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0),
|
||||
|
||||
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0),
|
||||
createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0),
|
||||
createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0),
|
||||
createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0),
|
||||
createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0),
|
||||
createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0),
|
||||
createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0),
|
||||
createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0),
|
||||
createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
Iterable<Row> results = runQuery(query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubqueryWithPostAggregatorsAndHaving()
|
||||
{
|
||||
final GroupByQuery subquery = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }"))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx_subagg", "index")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx_subpostagg",
|
||||
"+",
|
||||
Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"),
|
||||
new ConstantPostAggregator("thousand", 1000, 1000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setHavingSpec(
|
||||
new HavingSpec()
|
||||
{
|
||||
@Override
|
||||
public boolean eval(Row row)
|
||||
{
|
||||
return (row.getFloatMetric("idx_subpostagg") < 3800);
|
||||
}
|
||||
}
|
||||
)
|
||||
.addOrderByColumn("alias")
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
final GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(subquery)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("alias", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
new LongSumAggregatorFactory("rows", "rows"),
|
||||
new LongSumAggregatorFactory("idx", "idx_subpostagg")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000, 10000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0),
|
||||
createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0),
|
||||
createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0),
|
||||
createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0),
|
||||
createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0),
|
||||
createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0),
|
||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0),
|
||||
|
||||
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0),
|
||||
createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0),
|
||||
createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0),
|
||||
createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0),
|
||||
createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0),
|
||||
createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0),
|
||||
createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0),
|
||||
createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0),
|
||||
createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
Iterable<Row> results = runQuery(query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubqueryWithMultiColumnAggregators()
|
||||
{
|
||||
final GroupByQuery subquery = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setDimFilter(new JavaScriptDimFilter("provider", "function(dim){ return true; }"))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new DoubleSumAggregatorFactory("idx_subagg", "index"),
|
||||
new JavaScriptAggregatorFactory(
|
||||
"js_agg",
|
||||
Arrays.asList("index", "provider"),
|
||||
"function(current, index, dim){return current + index + dim.length;}",
|
||||
"function(){return 0;}",
|
||||
"function(a,b){return a + b;}"
|
||||
)
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx_subpostagg",
|
||||
"+",
|
||||
Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"),
|
||||
new ConstantPostAggregator("thousand", 1000, 1000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setHavingSpec(
|
||||
new HavingSpec()
|
||||
{
|
||||
@Override
|
||||
public boolean eval(Row row)
|
||||
{
|
||||
return (row.getFloatMetric("idx_subpostagg") < 3800);
|
||||
}
|
||||
}
|
||||
)
|
||||
.addOrderByColumn("alias")
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
final GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(subquery)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("alias", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
new LongSumAggregatorFactory("rows", "rows"),
|
||||
new LongSumAggregatorFactory("idx", "idx_subpostagg"),
|
||||
new DoubleSumAggregatorFactory("js_outer_agg", "js_agg")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000, 10000)
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setLimitSpec(
|
||||
new DefaultLimitSpec(
|
||||
Arrays.asList(
|
||||
new OrderByColumnSpec(
|
||||
"alias",
|
||||
OrderByColumnSpec.Direction.DESCENDING
|
||||
)
|
||||
),
|
||||
5
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.dayGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0, "js_outer_agg", 123.92274475097656),
|
||||
createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0, "js_outer_agg", 82.62254333496094),
|
||||
createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0, "js_outer_agg", 125.58358001708984),
|
||||
createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0, "js_outer_agg", 124.13470458984375),
|
||||
createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0, "js_outer_agg", 162.74722290039062)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
Iterable<Row> results = runQuery(query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
private Iterable<Row> runQuery(GroupByQuery query)
|
||||
{
|
||||
|
||||
|
|
|
@ -19,12 +19,16 @@
|
|||
|
||||
package io.druid.query.timeboundary;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.RetryQueryRunner;
|
||||
import io.druid.query.TableDataSource;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -32,8 +36,11 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -76,4 +83,89 @@ public class TimeBoundaryQueryRunnerTest
|
|||
Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime);
|
||||
Assert.assertEquals(new DateTime("2011-04-15T00:00:00.000Z"), maxTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTimeBoundaryMax()
|
||||
{
|
||||
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.bound(TimeBoundaryQuery.MAX_TIME)
|
||||
.build();
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
context.put(RetryQueryRunner.missingSegments, Lists.newArrayList());
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
runner.run(timeBoundaryQuery, context),
|
||||
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||
);
|
||||
TimeBoundaryResultValue val = results.iterator().next().getValue();
|
||||
DateTime minTime = val.getMinTime();
|
||||
DateTime maxTime = val.getMaxTime();
|
||||
|
||||
Assert.assertNull(minTime);
|
||||
Assert.assertEquals(new DateTime("2011-04-15T00:00:00.000Z"), maxTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testTimeBoundaryMin()
|
||||
{
|
||||
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.bound(TimeBoundaryQuery.MIN_TIME)
|
||||
.build();
|
||||
Map<String, Object> context = new MapMaker().makeMap();
|
||||
context.put(RetryQueryRunner.missingSegments, Lists.newArrayList());
|
||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||
runner.run(timeBoundaryQuery, context),
|
||||
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||
);
|
||||
TimeBoundaryResultValue val = results.iterator().next().getValue();
|
||||
DateTime minTime = val.getMinTime();
|
||||
DateTime maxTime = val.getMaxTime();
|
||||
|
||||
Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime);
|
||||
Assert.assertNull(maxTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeResults() throws Exception
|
||||
{
|
||||
List<Result<TimeBoundaryResultValue>> results = Arrays.asList(
|
||||
new Result<>(
|
||||
new DateTime(),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
"maxTime", "2012-01-01",
|
||||
"minTime", "2011-01-01"
|
||||
)
|
||||
)
|
||||
),
|
||||
new Result<>(
|
||||
new DateTime(),
|
||||
new TimeBoundaryResultValue(
|
||||
ImmutableMap.of(
|
||||
"maxTime", "2012-02-01",
|
||||
"minTime", "2011-01-01"
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
TimeBoundaryQuery query = new TimeBoundaryQuery(new TableDataSource("test"), null, null, null);
|
||||
Iterable<Result<TimeBoundaryResultValue>> actual = query.mergeResults(results);
|
||||
|
||||
Assert.assertTrue(actual.iterator().next().getValue().getMaxTime().equals(new DateTime("2012-02-01")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeResultsEmptyResults() throws Exception
|
||||
{
|
||||
List<Result<TimeBoundaryResultValue>> results = Lists.newArrayList();
|
||||
|
||||
TimeBoundaryQuery query = new TimeBoundaryQuery(new TableDataSource("test"), null, null, null);
|
||||
Iterable<Result<TimeBoundaryResultValue>> actual = query.mergeResults(results);
|
||||
|
||||
Assert.assertFalse(actual.iterator().hasNext());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -180,6 +180,7 @@ public class TestIndex
|
|||
new TimestampSpec("ts", "iso"),
|
||||
new DimensionsSpec(Arrays.asList(DIMENSIONS), null, null),
|
||||
"\t",
|
||||
"\u0001",
|
||||
Arrays.asList(COLUMNS)
|
||||
),
|
||||
null, null, null, null
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package io.druid.segment.data;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -47,7 +47,7 @@ public class CompressedLongsIndexedSupplierTest
|
|||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
Closeables.closeQuietly(indexed);
|
||||
CloseQuietly.close(indexed);
|
||||
indexed = null;
|
||||
supplier = null;
|
||||
vals = null;
|
||||
|
@ -56,7 +56,7 @@ public class CompressedLongsIndexedSupplierTest
|
|||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
Closeables.closeQuietly(indexed);
|
||||
CloseQuietly.close(indexed);
|
||||
}
|
||||
|
||||
private void setupSimple()
|
||||
|
@ -247,7 +247,7 @@ public class CompressedLongsIndexedSupplierTest
|
|||
stopLatch.await();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(indexed2);
|
||||
CloseQuietly.close(indexed2);
|
||||
}
|
||||
|
||||
if (failureHappened.get()) {
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
<parent>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.122-SNAPSHOT</version>
|
||||
<version>0.6.129-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -128,7 +128,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory<StringInputRowPa
|
|||
@Override
|
||||
public Firehose connect(StringInputRowParser firehoseParser) throws IOException
|
||||
{
|
||||
final StringInputRowParser stringParser = (StringInputRowParser) firehoseParser;
|
||||
final StringInputRowParser stringParser = firehoseParser;
|
||||
|
||||
ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory);
|
||||
Config lyraConfig = new Config()
|
||||
|
|
|
@ -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,7 +21,6 @@ package io.druid.storage.s3;
|
|||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
|
@ -95,9 +94,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
|||
try {
|
||||
s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path);
|
||||
|
||||
InputStream in = null;
|
||||
try {
|
||||
in = s3Obj.getDataInputStream();
|
||||
try (InputStream in = s3Obj.getDataInputStream()) {
|
||||
final String key = s3Obj.getKey();
|
||||
if (key.endsWith(".zip")) {
|
||||
CompressionUtils.unzip(in, outDir);
|
||||
|
@ -113,9 +110,6 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
|||
catch (IOException e) {
|
||||
throw new IOException(String.format("Problem decompressing object[%s]", s3Obj), e);
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
S3Utils.closeStreamsQuietly(s3Obj);
|
||||
|
@ -127,7 +121,8 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
|||
catch (Exception e) {
|
||||
try {
|
||||
FileUtils.deleteDirectory(outDir);
|
||||
} catch (IOException ioe) {
|
||||
}
|
||||
catch (IOException ioe) {
|
||||
log.warn(
|
||||
ioe,
|
||||
"Failed to remove output directory for segment[%s] after exception: %s",
|
||||
|
|
|
@ -18,8 +18,7 @@
|
|||
~ 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-server</artifactId>
|
||||
|
@ -29,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>
|
||||
|
|
|
@ -29,7 +29,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -37,6 +36,7 @@ import com.metamx.common.IAE;
|
|||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.RE;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
@ -266,7 +266,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
@Override
|
||||
public void cleanup(JsonParserIterator<T> iterFromMake)
|
||||
{
|
||||
Closeables.closeQuietly(iterFromMake);
|
||||
CloseQuietly.close(iterFromMake);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -309,7 +309,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
return false;
|
||||
}
|
||||
if (jp.getCurrentToken() == JsonToken.END_ARRAY) {
|
||||
Closeables.closeQuietly(jp);
|
||||
CloseQuietly.close(jp);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -23,10 +23,10 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
@ -112,7 +112,7 @@ public class Announcer
|
|||
started = false;
|
||||
|
||||
for (Map.Entry<String, PathChildrenCache> entry : listeners.entrySet()) {
|
||||
Closeables.closeQuietly(entry.getValue());
|
||||
CloseQuietly.close(entry.getValue());
|
||||
}
|
||||
|
||||
for (Map.Entry<String, ConcurrentMap<String, byte[]>> entry : announcements.entrySet()) {
|
||||
|
@ -353,7 +353,7 @@ public class Announcer
|
|||
cache.start();
|
||||
}
|
||||
catch (Exception e) {
|
||||
Closeables.closeQuietly(cache);
|
||||
CloseQuietly.close(cache);
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -406,6 +406,12 @@ public class DiscoveryModule implements Module
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ServiceInstance<T>> getAllInstances() throws Exception
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void noteError(ServiceInstance<T> tServiceInstance) {
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.curator.x.discovery.ServiceInstance;
|
|||
import org.apache.curator.x.discovery.ServiceProvider;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -62,6 +63,12 @@ public class ServerDiscoveryFactory
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ServiceInstance<T>> getAllInstances() throws Exception
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void noteError(ServiceInstance<T> tServiceInstance) {
|
||||
// do nothing
|
||||
|
|
|
@ -120,6 +120,14 @@ public class Initialization
|
|||
return retVal;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for testing only
|
||||
*/
|
||||
protected static void clearLoadedModules()
|
||||
{
|
||||
extensionsMap.clear();
|
||||
}
|
||||
|
||||
public synchronized static <T> Collection<T> getFromExtensions(ExtensionsConfig config, Class<T> clazz)
|
||||
{
|
||||
final TeslaAether aether = getAetherClient(config);
|
||||
|
|
|
@ -23,11 +23,11 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.parsers.ParseException;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.InputRow;
|
||||
|
@ -95,7 +95,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
|||
public void stop()
|
||||
{
|
||||
for (FireChief chief : chiefs.values()) {
|
||||
Closeables.closeQuietly(chief);
|
||||
CloseQuietly.close(chief);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -185,7 +185,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
|||
|
||||
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow;
|
||||
InputRow inputRow = null;
|
||||
try {
|
||||
try {
|
||||
inputRow = firehose.nextRow();
|
||||
|
@ -214,10 +214,11 @@ public class RealtimeManager implements QuerySegmentWalker
|
|||
}
|
||||
metrics.incrementProcessed();
|
||||
}
|
||||
catch (FormattedException e) {
|
||||
log.info(e, "unparseable line: %s", e.getDetails());
|
||||
catch (ParseException e) {
|
||||
if (inputRow != null) {
|
||||
log.error(e, "unparseable line: %s", inputRow);
|
||||
}
|
||||
metrics.incrementUnparseable();
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -237,7 +238,7 @@ public class RealtimeManager implements QuerySegmentWalker
|
|||
throw e;
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(firehose);
|
||||
CloseQuietly.close(firehose);
|
||||
if (normalExit) {
|
||||
plumber.finishJob();
|
||||
plumber = null;
|
||||
|
|
|
@ -31,18 +31,13 @@ import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg;
|
|||
import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter;
|
||||
import com.ircclouds.irc.api.state.IIRCState;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.ByteBufferInputRowParser;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.ParseSpec;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue