fixed merge conflicts

This commit is contained in:
jisookim0513 2014-07-07 11:26:11 -07:00
commit c4397bd65a
119 changed files with 1719 additions and 610 deletions

View File

@ -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>

View File

@ -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>

View File

@ -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);
}
}

View File

@ -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

View File

@ -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|

View File

@ -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:

View File

@ -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>

View File

@ -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>

View File

@ -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")));
}

View File

@ -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(

View File

@ -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>

View File

@ -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();

View File

@ -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>

View File

@ -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;
}
}
}

View File

@ -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
);

View File

@ -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);
}
}

View File

@ -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())) {

View File

@ -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>()
{

View File

@ -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();
}

View File

@ -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

View File

@ -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 + '\'' +
'}';
}
}

View File

@ -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;
}

View File

@ -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);

View File

@ -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()

View File

@ -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()
);
}
}

View File

@ -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")
)
);

View File

@ -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(

View File

@ -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>

View File

@ -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

View File

@ -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>

View File

@ -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
View File

@ -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>

View File

@ -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>

View File

@ -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...

View File

@ -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();
}
}
);

View File

@ -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);
}

View File

@ -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;

View File

@ -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,

View File

@ -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;
}
}

View File

@ -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();
/**

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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)
{

View File

@ -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;
}

View File

@ -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)
{

View File

@ -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)
{

View File

@ -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)
{

View File

@ -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());
}
}
}

View File

@ -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

View File

@ -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)
{

View File

@ -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;
}
}

View File

@ -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);
}
}
)

View File

@ -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);
}
}

View File

@ -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

View File

@ -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;
}

View File

@ -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);
}

View File

@ -41,6 +41,12 @@ public class NoopLimitSpec implements LimitSpec
return Functions.identity();
}
@Override
public LimitSpec merge(LimitSpec other)
{
return other;
}
@Override
public String toString()
{

View File

@ -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;
}
}

View File

@ -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);
}
};
}

View File

@ -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();
}

View File

@ -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) {

View File

@ -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();
}
}
}
}
);
}
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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:

View File

@ -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);

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -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();
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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();

View File

@ -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);
}
};
}
}

View File

@ -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

View File

@ -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));
}
}

View File

@ -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");

View File

@ -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)
{

View File

@ -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());
}
}

View File

@ -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

View File

@ -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()) {

View File

@ -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>

View File

@ -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()

View File

@ -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>

View File

@ -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",

View File

@ -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>

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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) {

View File

@ -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

View File

@ -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);

View File

@ -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;

View File

@ -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