trying to revert files that have overwritten changes

This commit is contained in:
jisookim0513 2014-09-26 12:38:04 -07:00
commit 43cc6283d3
133 changed files with 2289 additions and 718 deletions

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -71,7 +71,8 @@ public class DruidSecondaryModule implements Module
binder.install(new DruidGuiceExtensions()); binder.install(new DruidGuiceExtensions());
binder.bind(Properties.class).toInstance(properties); binder.bind(Properties.class).toInstance(properties);
binder.bind(ConfigurationObjectFactory.class).toInstance(factory); binder.bind(ConfigurationObjectFactory.class).toInstance(factory);
binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); // make objectMapper eager to ensure jackson gets setup with guice injection for JsonConfigurator
binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)).asEagerSingleton();
binder.bind(Validator.class).toInstance(validator); binder.bind(Validator.class).toInstance(validator);
binder.bind(JsonConfigurator.class).toInstance(jsonConfigurator); binder.bind(JsonConfigurator.class).toInstance(jsonConfigurator);
} }

View File

@ -58,6 +58,7 @@ DRUID_CP=${EXAMPLE_LOC}
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/../config/realtime DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/../config/realtime
#For the kit #For the kit
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/lib/* DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/lib/*
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/_global
DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/realtime DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/realtime
echo "Running command:" echo "Running command:"

View File

@ -0,0 +1,23 @@
# Extensions
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.7.0","io.druid.extensions:druid-kafka-seven:0.7.0","io.druid.extensions:druid-rabbitmq:0.7.0", "io.druid.extensions:druid-s3-extensions:0.7.0"]
# Zookeeper
druid.zk.service.host=localhost
# Metadata Storage
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
# Deep storage
druid.storage.type=local
druid.storage.storage.storageDirectory=/tmp/druid/localStorage
# Indexing service discovery
druid.selectors.indexing.serviceName=overlord
# Monitoring (disabled for examples)
# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
# Metrics logging (disabled for examples)
druid.emitter=noop

View File

@ -2,8 +2,6 @@ druid.host=localhost
druid.service=broker druid.service=broker
druid.port=8080 druid.port=8080
druid.zk.service.host=localhost # Bump these up only for faster nested groupBy
# Change these to make Druid faster
druid.processing.buffer.sizeBytes=100000000 druid.processing.buffer.sizeBytes=100000000
druid.processing.numThreads=1 druid.processing.numThreads=1

View File

@ -2,10 +2,4 @@ druid.host=localhost
druid.service=coordinator druid.service=coordinator
druid.port=8082 druid.port=8082
druid.zk.service.host=localhost
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.coordinator.startDelay=PT70s druid.coordinator.startDelay=PT70s

View File

@ -2,9 +2,7 @@ druid.host=localhost
druid.service=historical druid.service=historical
druid.port=8081 druid.port=8081
druid.zk.service.host=localhost druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.7.0"]
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.147"]
# Dummy read only AWS account (used to download example data) # Dummy read only AWS account (used to download example data)
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b

View File

@ -1,18 +1,8 @@
druid.host=localhost druid.host=localhost
druid.port=8087 druid.port=8080
druid.service=overlord druid.service=overlord
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.147"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M druid.indexer.queue.startDelay=PT0M
druid.indexer.runner.javaOpts="-server -Xmx256m" druid.indexer.runner.javaOpts="-server -Xmx256m"
druid.indexer.runner.startPort=8088
druid.indexer.fork.property.druid.processing.numThreads=1 druid.indexer.fork.property.druid.processing.numThreads=1
druid.indexer.fork.property.druid.computation.buffer.size=100000000 druid.indexer.fork.property.druid.computation.buffer.size=100000000

View File

@ -2,19 +2,11 @@ druid.host=localhost
druid.service=realtime druid.service=realtime
druid.port=8083 druid.port=8083
druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.147","io.druid.extensions:druid-kafka-seven:0.6.147","io.druid.extensions:druid-rabbitmq:0.6.147"]
# Change this config to db to hand off to the rest of the Druid cluster # Change this config to db to hand off to the rest of the Druid cluster
druid.publish.type=noop druid.publish.type=noop
# These configs are only required for real hand off
# druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
# druid.db.connector.user=druid
# druid.db.connector.password=diurd
druid.processing.buffer.sizeBytes=100000000 druid.processing.buffer.sizeBytes=100000000
druid.processing.numThreads=1 druid.processing.numThreads=1
druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor"] # Enable Real monitoring
# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor","io.druid.segment.realtime.RealtimeMetricsMonitor"]

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -27,7 +27,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -48,6 +48,7 @@ import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -241,7 +242,8 @@ public class ApproximateHistogramQueryTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
} }

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -38,12 +38,6 @@
<version>${project.parent.version}</version> <version>${project.parent.version}</version>
</dependency> </dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-common</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>java-util</artifactId> <artifactId>java-util</artifactId>
@ -97,7 +91,6 @@
</dependency> </dependency>
<!-- Tests --> <!-- Tests -->
<dependency> <dependency>
<groupId>junit</groupId> <groupId>junit</groupId>

View File

@ -47,7 +47,6 @@ import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec; import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization; import io.druid.initialization.Initialization;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.server.DruidNode; import io.druid.server.DruidNode;
import io.druid.timeline.DataSegment; import io.druid.timeline.DataSegment;

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -49,7 +49,7 @@ public class ForkingTaskRunnerConfig
@JsonProperty @JsonProperty
@Min(1024) @Min(1024)
@Max(65535) @Max(65535)
private int startPort = 8081; private int startPort = 8100;
@JsonProperty @JsonProperty
@NotNull @NotNull

View File

@ -31,7 +31,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.155-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

11
pom.xml
View File

@ -23,14 +23,14 @@
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<packaging>pom</packaging> <packaging>pom</packaging>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
<name>druid</name> <name>druid</name>
<description>druid</description> <description>druid</description>
<scm> <scm>
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection> <connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection> <developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
<url>http://www.github.com/metamx/druid</url> <url>http://www.github.com/metamx/druid</url>
<tag>druid-0.6.131-SNAPSHOT</tag> <tag>druid-0.7.0-SNAPSHOT</tag>
</scm> </scm>
<prerequisites> <prerequisites>
@ -75,7 +75,7 @@
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
<artifactId>emitter</artifactId> <artifactId>emitter</artifactId>
<version>0.2.11</version> <version>0.2.12</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.metamx</groupId> <groupId>com.metamx</groupId>
@ -385,7 +385,7 @@
<dependency> <dependency>
<groupId>net.jpountz.lz4</groupId> <groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId> <artifactId>lz4</artifactId>
<version>1.1.2</version> <version>1.2.0</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.google.protobuf</groupId> <groupId>com.google.protobuf</groupId>
@ -569,6 +569,9 @@
<groupId>org.apache.maven.plugins</groupId> <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-release-plugin</artifactId> <artifactId>maven-release-plugin</artifactId>
<version>2.5</version> <version>2.5</version>
<configuration>
<autoVersionSubmodules>true</autoVersionSubmodules>
</configuration>
</plugin> </plugin>
</plugins> </plugins>
</pluginManagement> </pluginManagement>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>
@ -82,6 +82,10 @@
<groupId>com.davekoelle</groupId> <groupId>com.davekoelle</groupId>
<artifactId>alphanum</artifactId> <artifactId>alphanum</artifactId>
</dependency> </dependency>
<dependency>
<groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId>
</dependency>
<!-- Tests --> <!-- Tests -->

View File

@ -33,6 +33,7 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.PropertiesModule; import io.druid.guice.PropertiesModule;
import io.druid.jackson.JacksonModule; import io.druid.jackson.JacksonModule;
import java.util.Arrays;
import java.util.List; import java.util.List;
/** /**
@ -44,7 +45,7 @@ public class GuiceInjectors
return Guice.createInjector( return Guice.createInjector(
new DruidGuiceExtensions(), new DruidGuiceExtensions(),
new JacksonModule(), new JacksonModule(),
new PropertiesModule("runtime.properties"), new PropertiesModule(Arrays.asList("global.runtime.properties", "runtime.properties")),
new ConfigModule(), new ConfigModule(),
new Module() new Module()
{ {
@ -63,7 +64,7 @@ public class GuiceInjectors
List<Module> theModules = Lists.newArrayList(); List<Module> theModules = Lists.newArrayList();
theModules.add(new DruidGuiceExtensions()); theModules.add(new DruidGuiceExtensions());
theModules.add(new JacksonModule()); theModules.add(new JacksonModule());
theModules.add(new PropertiesModule("runtime.properties")); theModules.add(new PropertiesModule(Arrays.asList("global.runtime.properties", "runtime.properties")));
theModules.add(new ConfigModule()); theModules.add(new ConfigModule());
theModules.add( theModules.add(
new Module() new Module()

View File

@ -17,10 +17,11 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.guice; package io.druid.guice;;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Module; import com.google.inject.Module;
import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.CloseQuietly;
@ -33,7 +34,7 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.io.Reader; import java.util.List;
import java.util.Properties; import java.util.Properties;
/** /**
@ -42,11 +43,11 @@ public class PropertiesModule implements Module
{ {
private static final Logger log = new Logger(PropertiesModule.class); private static final Logger log = new Logger(PropertiesModule.class);
private final String propertiesFile; private final List<String> propertiesFiles;
public PropertiesModule(String propertiesFile) public PropertiesModule(List<String> propertiesFiles)
{ {
this.propertiesFile = propertiesFile; this.propertiesFiles = propertiesFiles;
} }
@Override @Override
@ -58,6 +59,7 @@ public class PropertiesModule implements Module
Properties props = new Properties(fileProps); Properties props = new Properties(fileProps);
props.putAll(systemProps); props.putAll(systemProps);
for (String propertiesFile : propertiesFiles) {
InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile);
try { try {
if (stream == null) { if (stream == null) {
@ -69,8 +71,8 @@ public class PropertiesModule implements Module
if (stream != null) { if (stream != null) {
log.info("Loading properties from %s", propertiesFile); log.info("Loading properties from %s", propertiesFile);
try(Reader reader = new InputStreamReader(stream, Charsets.UTF_8)) { try {
fileProps.load(reader); fileProps.load(new InputStreamReader(stream, Charsets.UTF_8));
} }
catch (IOException e) { catch (IOException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -83,6 +85,7 @@ public class PropertiesModule implements Module
finally { finally {
CloseQuietly.close(stream); CloseQuietly.close(stream);
} }
}
binder.bind(Properties.class).toInstance(props); binder.bind(Properties.class).toInstance(props);
} }

View File

@ -70,14 +70,14 @@ public abstract class BaseQuery<T> implements Query<T>
} }
@Override @Override
public Sequence<T> run(QuerySegmentWalker walker) public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context)
{ {
return run(querySegmentSpec.lookup(this, walker)); return run(querySegmentSpec.lookup(this, walker), context);
} }
public Sequence<T> run(QueryRunner<T> runner) public Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context)
{ {
return runner.run(this); return runner.run(this, context);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import org.joda.time.DateTime;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -52,11 +53,11 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, Map<String, Object> context)
{ {
if (query.getContextBySegment(false)) { if (query.getContextBySegment(false)) {
final Sequence<T> baseSequence = base.run(query);
final Sequence<T> baseSequence = base.run(query, context);
final List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList()); final List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
return Sequences.simple( return Sequences.simple(
Arrays.asList( Arrays.asList(
@ -71,7 +72,6 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
) )
); );
} }
return base.run(query, context);
return base.run(query);
} }
} }

View File

@ -21,6 +21,8 @@ package io.druid.query;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import java.util.Map;
/** /**
*/ */
public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T> public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
@ -35,14 +37,14 @@ public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(Query<T> query) public Sequence<T> run(Query<T> query, Map<String, Object> context)
{ {
if (query.getContextBySegment(false)) { if (query.getContextBySegment(false)) {
return baseRunner.run(query); return baseRunner.run(query, context);
} }
return doRun(baseRunner, query); return doRun(baseRunner, query, context);
} }
protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query); protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, Object> context);
} }

View File

@ -39,6 +39,7 @@ import com.metamx.common.logger.Logger;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
@ -93,7 +94,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
final int priority = query.getContextPriority(0); final int priority = query.getContextPriority(0);
@ -124,7 +125,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
throw new ISE("Input is null?! How is this possible?!"); throw new ISE("Input is null?! How is this possible?!");
} }
Sequence<T> result = input.run(query); Sequence<T> result = input.run(query, context);
if (result == null) { if (result == null) {
throw new ISE("Got a null result! Segments are missing!"); throw new ISE("Got a null result! Segments are missing!");
} }

View File

@ -23,6 +23,9 @@ import com.google.common.base.Function;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import java.util.List;
import java.util.Map;
/** /**
*/ */
public class ConcatQueryRunner<T> implements QueryRunner<T> public class ConcatQueryRunner<T> implements QueryRunner<T>
@ -36,7 +39,7 @@ public class ConcatQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
return Sequences.concat( return Sequences.concat(
Sequences.map( Sequences.map(
@ -46,7 +49,7 @@ public class ConcatQueryRunner<T> implements QueryRunner<T>
@Override @Override
public Sequence<T> apply(final QueryRunner<T> input) public Sequence<T> apply(final QueryRunner<T> input)
{ {
return input.run(query); return input.run(query, context);
} }
} }
) )

View File

@ -22,12 +22,15 @@ package io.druid.query;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.query.aggregation.MetricManipulatorFns; import io.druid.query.aggregation.MetricManipulatorFns;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -46,7 +49,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, Map<String, Object> context)
{ {
final boolean isBySegment = query.getContextBySegment(false); final boolean isBySegment = query.getContextBySegment(false);
final boolean shouldFinalize = query.getContextFinalize(true); final boolean shouldFinalize = query.getContextFinalize(true);
@ -73,9 +76,14 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public T apply(@Nullable T input) public T apply(T input)
{ {
Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input; Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input;
if (input == null) {
throw new ISE("Cannot have a null result!");
}
BySegmentResultValueClass<T> resultsClass = result.getValue(); BySegmentResultValueClass<T> resultsClass = result.getValue();
return (T) new Result<BySegmentResultValueClass>( return (T) new Result<BySegmentResultValueClass>(
@ -94,7 +102,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
return Sequences.map( return Sequences.map(
baseRunner.run(queryToRun), baseRunner.run(queryToRun, context),
finalizerFn finalizerFn
); );

View File

@ -41,6 +41,7 @@ import io.druid.query.groupby.GroupByQueryHelper;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
@ -70,7 +71,7 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> queryParam) public Sequence<T> run(final Query<T> queryParam, final Map<String, Object> context)
{ {
final GroupByQuery query = (GroupByQuery) queryParam; final GroupByQuery query = (GroupByQuery) queryParam;
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( final Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
@ -101,10 +102,10 @@ public class GroupByParallelQueryRunner<T> implements QueryRunner<T>
{ {
try { try {
if (bySegment) { if (bySegment) {
input.run(queryParam) input.run(queryParam, context)
.accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs);
} else { } else {
input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); input.run(queryParam, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
} }
return null; return null;

View File

@ -33,6 +33,7 @@ import javax.annotation.Nullable;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -48,10 +49,10 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
if (period.getMillis() == 0) { if (period.getMillis() == 0) {
return baseRunner.run(query); return baseRunner.run(query, context);
} }
return Sequences.concat( return Sequences.concat(
@ -74,7 +75,8 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
public Sequence<T> apply(Interval singleInterval) public Sequence<T> apply(Interval singleInterval)
{ {
return baseRunner.run( return baseRunner.run(
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))) query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))),
context
); );
} }
} }

View File

@ -28,6 +28,8 @@ import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import java.io.IOException; import java.io.IOException;
import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -66,7 +68,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
final ServiceMetricEvent.Builder builder = builderFn.apply(query); final ServiceMetricEvent.Builder builder = builderFn.apply(query);
String queryId = query.getId(); String queryId = query.getId();
@ -84,7 +86,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
try { try {
retVal = queryRunner.run(query).accumulate(outType, accumulator); retVal = queryRunner.run(query, context).accumulate(outType, accumulator);
} }
catch (RuntimeException e) { catch (RuntimeException e) {
builder.setUser10("failed"); builder.setUser10("failed");
@ -114,7 +116,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
try { try {
retVal = queryRunner.run(query).toYielder(initValue, accumulator); retVal = queryRunner.run(query, context).toYielder(initValue, accumulator);
} }
catch (RuntimeException e) { catch (RuntimeException e) {
builder.setUser10("failed"); builder.setUser10("failed");

View File

@ -22,12 +22,15 @@ package io.druid.query;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import java.util.List;
import java.util.Map;
/** /**
*/ */
public class NoopQueryRunner<T> implements QueryRunner<T> public class NoopQueryRunner<T> implements QueryRunner<T>
{ {
@Override @Override
public Sequence<T> run(Query query) public Sequence<T> run(Query<T> query, Map<String, Object> context)
{ {
return Sequences.empty(); return Sequences.empty();
} }

View File

@ -62,9 +62,9 @@ public interface Query<T>
public String getType(); public String getType();
public Sequence<T> run(QuerySegmentWalker walker); public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context);
public Sequence<T> run(QueryRunner<T> runner); public Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context);
public List<Interval> getIntervals(); public List<Interval> getIntervals();

View File

@ -21,9 +21,11 @@ package io.druid.query;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import java.util.Map;
/** /**
*/ */
public interface QueryRunner<T> public interface QueryRunner<T>
{ {
public Sequence<T> run(Query<T> query); public Sequence<T> run(Query<T> query, Map<String, Object> context);
} }

View File

@ -40,12 +40,15 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
/** /**
* This method doesn't belong here, but it's here for now just to make it work. * This method doesn't belong here, but it's here for now just to make it work.
* *
* @param seqOfSequences * @param seqOfSequences sequence of sequences to be merged
* *
* @return * @return the sequence of merged results
*/ */
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences); public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
public abstract Sequence<ResultType> mergeSequencesUnordered(Sequence<Sequence<ResultType>> seqOfSequences);
public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
public abstract Function<ResultType, ResultType> makePreComputeManipulatorFn( public abstract Function<ResultType, ResultType> makePreComputeManipulatorFn(

View File

@ -25,6 +25,8 @@ import com.metamx.common.guava.Sequence;
import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.ReferenceCountingSegment;
import java.io.Closeable; import java.io.Closeable;
import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -43,11 +45,11 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, Map<String, Object> context)
{ {
final Closeable closeable = adapter.increment(); final Closeable closeable = adapter.increment();
try { try {
final Sequence<T> baseSequence = factory.createRunner(adapter).run(query); final Sequence<T> baseSequence = factory.createRunner(adapter).run(query, context);
return new ResourceClosingSequence<T>(baseSequence, closeable); return new ResourceClosingSequence<T>(baseSequence, closeable);
} }

View File

@ -24,6 +24,9 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.guava.nary.BinaryFn;
import io.druid.common.guava.CombiningSequence; import io.druid.common.guava.CombiningSequence;
import java.util.List;
import java.util.Map;
/** /**
*/ */
public abstract class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRunner<T> public abstract class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRunner<T>
@ -36,9 +39,9 @@ public abstract class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRu
} }
@Override @Override
public Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query) public Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, Object> context)
{ {
return CombiningSequence.create(baseRunner.run(query), makeOrdering(query), createMergeFn(query)); return CombiningSequence.create(baseRunner.run(query, context), makeOrdering(query), createMergeFn(query));
} }
protected abstract Ordering<T> makeOrdering(Query<T> query); protected abstract Ordering<T> makeOrdering(Query<T> query);

View File

@ -0,0 +1,113 @@
/*
* 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.query;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.YieldingAccumulator;
import com.metamx.common.guava.YieldingSequenceBase;
import io.druid.query.spec.MultipleSpecificSegmentSpec;
import io.druid.segment.SegmentMissingException;
import java.util.List;
import java.util.Map;
public class RetryQueryRunner<T> implements QueryRunner<T>
{
public static String MISSING_SEGMENTS_KEY = "missingSegments";
private final QueryRunner<T> baseRunner;
private final RetryQueryRunnerConfig config;
private final ObjectMapper jsonMapper;
public RetryQueryRunner(
QueryRunner<T> baseRunner,
RetryQueryRunnerConfig config,
ObjectMapper jsonMapper
)
{
this.baseRunner = baseRunner;
this.config = config;
this.jsonMapper = jsonMapper;
}
@Override
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{
final Sequence<T> returningSeq = baseRunner.run(query, context);
return new YieldingSequenceBase<T>()
{
@Override
public <OutType> Yielder<OutType> toYielder(
OutType initValue, YieldingAccumulator<OutType, T> accumulator
)
{
Yielder<OutType> yielder = returningSeq.toYielder(initValue, accumulator);
final List<SegmentDescriptor> missingSegments = getMissingSegments(context);
if (missingSegments.isEmpty()) {
return yielder;
}
for (int i = 0; i < config.numTries(); i++) {
context.put(MISSING_SEGMENTS_KEY, Lists.newArrayList());
final Query<T> retryQuery = query.withQuerySegmentSpec(
new MultipleSpecificSegmentSpec(
missingSegments
)
);
yielder = baseRunner.run(retryQuery, context).toYielder(initValue, accumulator);
if (getMissingSegments(context).isEmpty()) {
break;
}
}
final List<SegmentDescriptor> finalMissingSegs = getMissingSegments(context);
if (!config.returnPartialResults() && !finalMissingSegs.isEmpty()) {
throw new SegmentMissingException("No results found for segments[%s]", finalMissingSegs);
}
return yielder;
}
};
}
private List<SegmentDescriptor> getMissingSegments(final Map<String, Object> context)
{
final Object maybeMissingSegments = context.get(MISSING_SEGMENTS_KEY);
if (maybeMissingSegments == null) {
return Lists.newArrayList();
}
return jsonMapper.convertValue(
maybeMissingSegments,
new TypeReference<List<SegmentDescriptor>>()
{
}
);
}
}

View File

@ -0,0 +1,33 @@
/*
* 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.query;
import com.fasterxml.jackson.annotation.JsonProperty;
public class RetryQueryRunnerConfig
{
@JsonProperty
private int numTries = 0;
@JsonProperty
private boolean returnPartialResults = false;
public int numTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
}

View File

@ -23,6 +23,9 @@ package io.druid.query;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import java.util.List;
import java.util.Map;
/** /**
* If there's a subquery, run it instead of the outer query * If there's a subquery, run it instead of the outer query
*/ */
@ -36,13 +39,13 @@ public class SubqueryQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, Map<String, Object> context)
{ {
DataSource dataSource = query.getDataSource(); DataSource dataSource = query.getDataSource();
if (dataSource instanceof QueryDataSource) { if (dataSource instanceof QueryDataSource) {
return run((Query<T>) ((QueryDataSource) dataSource).getQuery()); return run((Query<T>) ((QueryDataSource) dataSource).getQuery(), context);
} else { } else {
return baseRunner.run(query); return baseRunner.run(query, context);
} }
} }
} }

View File

@ -33,6 +33,8 @@ import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map;
/** /**
* TimewarpOperator is an example post-processing operator that maps current time * TimewarpOperator is an example post-processing operator that maps current time
@ -79,7 +81,7 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
return new QueryRunner<T>() return new QueryRunner<T>()
{ {
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
final long offset = computeOffset(now); final long offset = computeOffset(now);
@ -90,7 +92,8 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
); );
return Sequences.map( return Sequences.map(
baseRunner.run( baseRunner.run(
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))) query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))),
context
), ),
new Function<T, T>() new Function<T, T>()
{ {

View File

@ -26,7 +26,8 @@ import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import java.util.ArrayList; import java.util.List;
import java.util.Map;
public class UnionQueryRunner<T> implements QueryRunner<T> public class UnionQueryRunner<T> implements QueryRunner<T>
{ {
@ -43,11 +44,11 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> query) public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
{ {
DataSource dataSource = query.getDataSource(); DataSource dataSource = query.getDataSource();
if (dataSource instanceof UnionDataSource) { if (dataSource instanceof UnionDataSource) {
return toolChest.mergeSequences( return toolChest.mergeSequencesUnordered(
Sequences.simple( Sequences.simple(
Lists.transform( Lists.transform(
((UnionDataSource) dataSource).getDataSources(), ((UnionDataSource) dataSource).getDataSources(),
@ -57,7 +58,8 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
public Sequence<T> apply(DataSource singleSource) public Sequence<T> apply(DataSource singleSource)
{ {
return baseRunner.run( return baseRunner.run(
query.withDataSource(singleSource) query.withDataSource(singleSource),
context
); );
} }
} }
@ -65,7 +67,7 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
) )
); );
} else { } else {
return baseRunner.run(query); return baseRunner.run(query, context);
} }
} }

View File

@ -106,21 +106,21 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
return new QueryRunner<Row>() return new QueryRunner<Row>()
{ {
@Override @Override
public Sequence<Row> run(Query<Row> input) public Sequence<Row> run(Query<Row> input, Map<String, Object> context)
{ {
if (input.getContextBySegment(false)) { if (input.getContextBySegment(false)) {
return runner.run(input); return runner.run(input, context);
} }
if (Boolean.valueOf(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); return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, context);
} }
return runner.run(input); return runner.run(input, context);
} }
}; };
} }
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner) private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, Object> context)
{ {
// If there's a subquery, merge subquery results and then apply the aggregator // If there's a subquery, merge subquery results and then apply the aggregator
final DataSource dataSource = query.getDataSource(); final DataSource dataSource = query.getDataSource();
@ -132,7 +132,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
catch (ClassCastException e) { catch (ClassCastException e) {
throw new UnsupportedOperationException("Subqueries must be of type 'group by'"); throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
} }
final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner); final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, context);
final List<AggregatorFactory> aggs = Lists.newArrayList(); final List<AggregatorFactory> aggs = Lists.newArrayList();
for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
aggs.addAll(aggregatorFactory.getRequiredColumns()); aggs.addAll(aggregatorFactory.getRequiredColumns());
@ -154,7 +154,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
); );
return outerQuery.applyLimit(engine.process(outerQuery, adapter)); return outerQuery.applyLimit(engine.process(outerQuery, adapter));
} else { } else {
return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query)))); return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query, context))));
} }
} }

View File

@ -49,6 +49,7 @@ import io.druid.segment.StorageAdapter;
import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndex;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -105,7 +106,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
return new QueryRunner<Row>() return new QueryRunner<Row>()
{ {
@Override @Override
public Sequence<Row> run(final Query<Row> query) public Sequence<Row> run(final Query<Row> query, final Map<String, Object> context)
{ {
final GroupByQuery queryParam = (GroupByQuery) query; final GroupByQuery queryParam = (GroupByQuery) query;
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper final Pair<IncrementalIndex, Accumulator<IncrementalIndex, Row>> indexAccumulatorPair = GroupByQueryHelper
@ -124,13 +125,13 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
public Void call() throws Exception public Void call() throws Exception
{ {
if (bySegment) { if (bySegment) {
input.run(queryParam) input.run(queryParam, context)
.accumulate( .accumulate(
bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.lhs,
bySegmentAccumulatorPair.rhs bySegmentAccumulatorPair.rhs
); );
} else { } else {
input.run(query).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); input.run(query, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
} }
return null; return null;
@ -197,7 +198,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
} }
@Override @Override
public Sequence<Row> run(Query<Row> input) public Sequence<Row> run(Query<Row> input, Map<String, Object> context)
{ {
if (!(input instanceof GroupByQuery)) { if (!(input instanceof GroupByQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class);

View File

@ -77,7 +77,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
return new QueryRunner<SegmentAnalysis>() return new QueryRunner<SegmentAnalysis>()
{ {
@Override @Override
public Sequence<SegmentAnalysis> run(Query<SegmentAnalysis> inQ) public Sequence<SegmentAnalysis> run(Query<SegmentAnalysis> inQ, Map<String, Object> context)
{ {
SegmentMetadataQuery query = (SegmentMetadataQuery) inQ; SegmentMetadataQuery query = (SegmentMetadataQuery) inQ;
@ -136,7 +136,10 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
return new QueryRunner<SegmentAnalysis>() return new QueryRunner<SegmentAnalysis>()
{ {
@Override @Override
public Sequence<SegmentAnalysis> run(final Query<SegmentAnalysis> query) public Sequence<SegmentAnalysis> run(
final Query<SegmentAnalysis> query,
final Map<String, Object> context
)
{ {
final int priority = query.getContextPriority(0); final int priority = query.getContextPriority(0);
ListenableFuture<Sequence<SegmentAnalysis>> future = queryExecutor.submit( ListenableFuture<Sequence<SegmentAnalysis>> future = queryExecutor.submit(
@ -145,7 +148,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
@Override @Override
public Sequence<SegmentAnalysis> call() throws Exception public Sequence<SegmentAnalysis> call() throws Exception
{ {
return input.run(query); return input.run(query, context);
} }
} }
); );

View File

@ -286,7 +286,10 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
} }
@Override @Override
public Sequence<Result<SearchResultValue>> run(Query<Result<SearchResultValue>> input) public Sequence<Result<SearchResultValue>> run(
Query<Result<SearchResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof SearchQuery)) { if (!(input instanceof SearchQuery)) {
throw new ISE("Can only handle [%s], got [%s]", SearchQuery.class, input.getClass()); throw new ISE("Can only handle [%s], got [%s]", SearchQuery.class, input.getClass());
@ -294,13 +297,13 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
final SearchQuery query = (SearchQuery) input; final SearchQuery query = (SearchQuery) input;
if (query.getLimit() < config.getMaxSearchLimit()) { if (query.getLimit() < config.getMaxSearchLimit()) {
return runner.run(query); return runner.run(query, context);
} }
final boolean isBySegment = query.getContextBySegment(false); final boolean isBySegment = query.getContextBySegment(false);
return Sequences.map( return Sequences.map(
runner.run(query.withLimit(config.getMaxSearchLimit())), runner.run(query.withLimit(config.getMaxSearchLimit()), context),
new Function<Result<SearchResultValue>, Result<SearchResultValue>>() new Function<Result<SearchResultValue>, Result<SearchResultValue>>()
{ {
@Override @Override

View File

@ -68,7 +68,10 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
} }
@Override @Override
public Sequence<Result<SearchResultValue>> run(final Query<Result<SearchResultValue>> input) public Sequence<Result<SearchResultValue>> run(
final Query<Result<SearchResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof SearchQuery)) { if (!(input instanceof SearchQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class);

View File

@ -34,6 +34,8 @@ import io.druid.query.QueryWatcher;
import io.druid.query.Result; import io.druid.query.Result;
import io.druid.segment.Segment; import io.druid.segment.Segment;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
/** /**
@ -91,7 +93,10 @@ public class SelectQueryRunnerFactory
} }
@Override @Override
public Sequence<Result<SelectResultValue>> run(Query<Result<SelectResultValue>> input) public Sequence<Result<SelectResultValue>> run(
Query<Result<SelectResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof SelectQuery)) { if (!(input instanceof SelectQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SelectQuery.class);

View File

@ -20,14 +20,21 @@
package io.druid.query.spec; package io.druid.query.spec;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.guava.Yielder; import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingAccumulator;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.RetryQueryRunner;
import io.druid.query.SegmentDescriptor;
import io.druid.segment.SegmentMissingException;
import java.io.IOException; import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
/** /**
@ -35,11 +42,11 @@ import java.util.concurrent.Callable;
public class SpecificSegmentQueryRunner<T> implements QueryRunner<T> public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
{ {
private final QueryRunner<T> base; private final QueryRunner<T> base;
private final QuerySegmentSpec specificSpec; private final SpecificSegmentSpec specificSpec;
public SpecificSegmentQueryRunner( public SpecificSegmentQueryRunner(
QueryRunner<T> base, QueryRunner<T> base,
QuerySegmentSpec specificSpec SpecificSegmentSpec specificSpec
) )
{ {
this.base = base; this.base = base;
@ -47,7 +54,7 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public Sequence<T> run(final Query<T> input) public Sequence<T> run(final Query<T> input, final Map<String, Object> context)
{ {
final Query<T> query = input.withQuerySegmentSpec(specificSpec); final Query<T> query = input.withQuerySegmentSpec(specificSpec);
@ -55,14 +62,29 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
final String currThreadName = currThread.getName(); final String currThreadName = currThread.getName();
final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals()); final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals());
final Sequence<T> baseSequence = doNamed(currThread, currThreadName, newName, new Callable<Sequence<T>>() final Sequence<T> baseSequence = doNamed(
currThread, currThreadName, newName, new Callable<Sequence<T>>()
{ {
@Override @Override
public Sequence<T> call() throws Exception public Sequence<T> call() throws Exception
{ {
return base.run(query); Sequence<T> returningSeq;
try {
returningSeq = base.run(query, context);
} }
}); catch (SegmentMissingException e) {
List<SegmentDescriptor> missingSegments = (List<SegmentDescriptor>) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY);
if (missingSegments == null) {
missingSegments = Lists.newArrayList();
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, missingSegments);
}
missingSegments.add(specificSpec.getDescriptor());
returningSeq = Sequences.empty();
}
return returningSeq;
}
}
);
return new Sequence<T>() return new Sequence<T>()
{ {
@ -82,7 +104,10 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
} }
@Override @Override
public <OutType> Yielder<OutType> toYielder(final OutType initValue, final YieldingAccumulator<OutType, T> accumulator) public <OutType> Yielder<OutType> toYielder(
final OutType initValue,
final YieldingAccumulator<OutType, T> accumulator
)
{ {
return doItNamed( return doItNamed(
new Callable<Yielder<OutType>>() new Callable<Yielder<OutType>>()

View File

@ -52,6 +52,8 @@ public class SpecificSegmentSpec implements QuerySegmentSpec
return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor));
} }
public SegmentDescriptor getDescriptor() { return descriptor; }
@Override @Override
public boolean equals(Object o) public boolean equals(Object o)
{ {

View File

@ -45,6 +45,7 @@ import org.joda.time.DateTime;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -95,13 +96,13 @@ public class TimeBoundaryQueryQueryToolChest
{ {
@Override @Override
protected Sequence<Result<TimeBoundaryResultValue>> doRun( protected Sequence<Result<TimeBoundaryResultValue>> doRun(
QueryRunner<Result<TimeBoundaryResultValue>> baseRunner, Query<Result<TimeBoundaryResultValue>> input QueryRunner<Result<TimeBoundaryResultValue>> baseRunner, Query<Result<TimeBoundaryResultValue>> input, Map<String, Object> context
) )
{ {
TimeBoundaryQuery query = (TimeBoundaryQuery) input; TimeBoundaryQuery query = (TimeBoundaryQuery) input;
return Sequences.simple( return Sequences.simple(
query.mergeResults( query.mergeResults(
Sequences.toList(baseRunner.run(query), Lists.<Result<TimeBoundaryResultValue>>newArrayList()) Sequences.toList(baseRunner.run(query, context), Lists.<Result<TimeBoundaryResultValue>>newArrayList())
) )
); );
} }

View File

@ -35,6 +35,8 @@ import io.druid.segment.StorageAdapter;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
/** /**
@ -83,7 +85,10 @@ public class TimeBoundaryQueryRunnerFactory
} }
@Override @Override
public Sequence<Result<TimeBoundaryResultValue>> run(Query<Result<TimeBoundaryResultValue>> input) public Sequence<Result<TimeBoundaryResultValue>> run(
Query<Result<TimeBoundaryResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof TimeBoundaryQuery)) { if (!(input instanceof TimeBoundaryQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeBoundaryQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeBoundaryQuery.class);

View File

@ -20,13 +20,13 @@
package io.druid.query.timeseries; package io.druid.query.timeseries;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import io.druid.query.QueryRunnerHelper; import io.druid.query.QueryRunnerHelper;
import io.druid.query.Result; import io.druid.query.Result;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.Cursor; import io.druid.segment.Cursor;
import io.druid.segment.SegmentMissingException;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;
import io.druid.segment.filter.Filters; import io.druid.segment.filter.Filters;
@ -39,7 +39,7 @@ public class TimeseriesQueryEngine
public Sequence<Result<TimeseriesResultValue>> process(final TimeseriesQuery query, final StorageAdapter adapter) public Sequence<Result<TimeseriesResultValue>> process(final TimeseriesQuery query, final StorageAdapter adapter)
{ {
if (adapter == null) { if (adapter == null) {
throw new ISE( throw new SegmentMissingException(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
); );
} }

View File

@ -32,6 +32,8 @@ import io.druid.query.Result;
import io.druid.segment.Segment; import io.druid.segment.Segment;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
/** /**
@ -89,7 +91,10 @@ public class TimeseriesQueryRunnerFactory
} }
@Override @Override
public Sequence<Result<TimeseriesResultValue>> run(Query<Result<TimeseriesResultValue>> input) public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof TimeseriesQuery)) { if (!(input instanceof TimeseriesQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class);

View File

@ -22,8 +22,6 @@ package io.druid.query.topn;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Predicates; import com.google.common.base.Predicates;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
@ -34,11 +32,11 @@ import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.segment.Capabilities; import io.druid.segment.Capabilities;
import io.druid.segment.Cursor; import io.druid.segment.Cursor;
import io.druid.segment.SegmentMissingException;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;
import io.druid.segment.filter.Filters; import io.druid.segment.filter.Filters;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.sql.rowset.Predicate;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
@ -58,7 +56,7 @@ public class TopNQueryEngine
public Sequence<Result<TopNResultValue>> query(final TopNQuery query, final StorageAdapter adapter) public Sequence<Result<TopNResultValue>> query(final TopNQuery query, final StorageAdapter adapter)
{ {
if (adapter == null) { if (adapter == null) {
throw new ISE( throw new SegmentMissingException(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
); );
} }

View File

@ -411,7 +411,10 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
} }
@Override @Override
public Sequence<Result<TopNResultValue>> run(Query<Result<TopNResultValue>> input) public Sequence<Result<TopNResultValue>> run(
Query<Result<TopNResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof TopNQuery)) { if (!(input instanceof TopNQuery)) {
throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass()); throw new ISE("Can only handle [%s], got [%s]", TopNQuery.class, input.getClass());
@ -419,13 +422,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
final TopNQuery query = (TopNQuery) input; final TopNQuery query = (TopNQuery) input;
if (query.getThreshold() > minTopNThreshold) { if (query.getThreshold() > minTopNThreshold) {
return runner.run(query); return runner.run(query, context);
} }
final boolean isBySegment = query.getContextBySegment(false); final boolean isBySegment = query.getContextBySegment(false);
return Sequences.map( return Sequences.map(
runner.run(query.withThreshold(minTopNThreshold)), runner.run(query.withThreshold(minTopNThreshold), context),
new Function<Result<TopNResultValue>, Result<TopNResultValue>>() new Function<Result<TopNResultValue>, Result<TopNResultValue>>()
{ {
@Override @Override

View File

@ -21,7 +21,6 @@ package io.druid.query.topn;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.guava.BaseSequence;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import io.druid.collections.StupidPool; import io.druid.collections.StupidPool;
import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Global;
@ -35,7 +34,8 @@ import io.druid.query.Result;
import io.druid.segment.Segment; import io.druid.segment.Segment;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Iterator; import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
/** /**
@ -65,7 +65,10 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory<Result<TopNRes
return new QueryRunner<Result<TopNResultValue>>() return new QueryRunner<Result<TopNResultValue>>()
{ {
@Override @Override
public Sequence<Result<TopNResultValue>> run(Query<Result<TopNResultValue>> input) public Sequence<Result<TopNResultValue>> run(
Query<Result<TopNResultValue>> input,
Map<String, Object> context
)
{ {
if (!(input instanceof TopNQuery)) { if (!(input instanceof TopNQuery)) {
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TopNQuery.class); throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TopNQuery.class);

View File

@ -21,6 +21,7 @@ package io.druid.segment;
import com.google.common.io.Files; import com.google.common.io.Files;
import io.druid.segment.data.CompressedFloatsSupplierSerializer; import io.druid.segment.data.CompressedFloatsSupplierSerializer;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.IOPeon; import io.druid.segment.data.IOPeon;
import java.io.File; import java.io.File;
@ -51,7 +52,8 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer
public void open() throws IOException public void open() throws IOException
{ {
writer = CompressedFloatsSupplierSerializer.create( writer = CompressedFloatsSupplierSerializer.create(
ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER,
CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
); );
writer.open(); writer.open();

View File

@ -54,6 +54,7 @@ import io.druid.query.aggregation.ToLowerCaseAggregatorFactory;
import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.ByteBufferWriter;
import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedLongsSupplierSerializer;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.ConciseCompressedIndexedInts;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.GenericIndexedWriter;
@ -594,7 +595,8 @@ public class IndexMerger
Iterable<Rowboat> theRows = rowMergerFn.apply(boats); Iterable<Rowboat> theRows = rowMergerFn.apply(boats);
CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create(
ioPeon, "little_end_time", IndexIO.BYTE_ORDER ioPeon, "little_end_time", IndexIO.BYTE_ORDER,
CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY
); );
timeWriter.open(); timeWriter.open();

View File

@ -0,0 +1,29 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment;
import com.metamx.common.ISE;
public class SegmentMissingException extends ISE
{
public SegmentMissingException(String formatText, Object... arguments) {
super(String.format(formatText, arguments));
}
}

View File

@ -28,14 +28,14 @@ import java.nio.FloatBuffer;
/** /**
*/ */
public class CompressedFloatBufferObjectStrategy extends CompressedObjectStrategy<FloatBuffer> public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<FloatBuffer>
{ {
public static CompressedFloatBufferObjectStrategy getBufferForOrder(ByteOrder order) public static CompressedFloatBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{ {
return new CompressedFloatBufferObjectStrategy(order); return new CompressedFloatBufferObjectStrategy(order, compression, sizePer);
} }
private CompressedFloatBufferObjectStrategy(final ByteOrder order) private CompressedFloatBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{ {
super( super(
order, order,
@ -64,7 +64,9 @@ public class CompressedFloatBufferObjectStrategy extends CompressedObjectStrateg
{ {
return into.asFloatBuffer().put(from); return into.asFloatBuffer().put(from);
} }
} },
compression,
sizePer
); );
} }
} }

View File

@ -40,22 +40,26 @@ import java.util.Iterator;
*/ */
public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats> public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
{ {
public static final byte version = 0x1; public static final byte LZF_VERSION = 0x1;
public static final byte version = 0x2;
public static final int MAX_FLOATS_IN_BUFFER = (0xFFFF >> 2); public static final int MAX_FLOATS_IN_BUFFER = (0xFFFF >> 2);
private final int totalSize; private final int totalSize;
private final int sizePer; private final int sizePer;
private final GenericIndexed<ResourceHolder<FloatBuffer>> baseFloatBuffers; private final GenericIndexed<ResourceHolder<FloatBuffer>> baseFloatBuffers;
private final CompressedObjectStrategy.CompressionStrategy compression;
CompressedFloatsIndexedSupplier( CompressedFloatsIndexedSupplier(
int totalSize, int totalSize,
int sizePer, int sizePer,
GenericIndexed<ResourceHolder<FloatBuffer>> baseFloatBuffers GenericIndexed<ResourceHolder<FloatBuffer>> baseFloatBuffers,
CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
this.totalSize = totalSize; this.totalSize = totalSize;
this.sizePer = sizePer; this.sizePer = sizePer;
this.baseFloatBuffers = baseFloatBuffers; this.baseFloatBuffers = baseFloatBuffers;
this.compression = compression;
} }
public int size() public int size()
@ -151,7 +155,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
public long getSerializedSize() public long getSerializedSize()
{ {
return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4; return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4 + 1;
} }
public void writeToChannel(WritableByteChannel channel) throws IOException public void writeToChannel(WritableByteChannel channel) throws IOException
@ -159,6 +163,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
channel.write(ByteBuffer.wrap(new byte[]{version})); channel.write(ByteBuffer.wrap(new byte[]{version}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
baseFloatBuffers.writeToChannel(channel); baseFloatBuffers.writeToChannel(channel);
} }
@ -167,7 +172,8 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
return new CompressedFloatsIndexedSupplier( return new CompressedFloatsIndexedSupplier(
totalSize, totalSize,
sizePer, sizePer,
GenericIndexed.fromIterable(baseFloatBuffers, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) GenericIndexed.fromIterable(baseFloatBuffers, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
compression
); );
} }
@ -191,23 +197,53 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
byte versionFromBuffer = buffer.get(); byte versionFromBuffer = buffer.get();
if (versionFromBuffer == version) { if (versionFromBuffer == version) {
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();
final CompressedObjectStrategy.CompressionStrategy compression =
CompressedObjectStrategy.CompressionStrategy.forId(buffer.get());
return new CompressedFloatsIndexedSupplier( return new CompressedFloatsIndexedSupplier(
buffer.getInt(), totalSize,
buffer.getInt(), sizePer,
GenericIndexed.read(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order)) GenericIndexed.read(
buffer,
CompressedFloatBufferObjectStrategy.getBufferForOrder(
order,
compression,
sizePer
)
),
compression
);
} else if (versionFromBuffer == LZF_VERSION) {
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF;
return new CompressedFloatsIndexedSupplier(
totalSize,
sizePer,
GenericIndexed.read(
buffer,
CompressedFloatBufferObjectStrategy.getBufferForOrder(
order,
compression,
sizePer
)
),
compression
); );
} }
throw new IAE("Unknown version[%s]", versionFromBuffer); throw new IAE("Unknown version[%s]", versionFromBuffer);
} }
public static CompressedFloatsIndexedSupplier fromFloatBuffer(FloatBuffer buffer, final ByteOrder order) public static CompressedFloatsIndexedSupplier fromFloatBuffer(FloatBuffer buffer, final ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression)
{ {
return fromFloatBuffer(buffer, MAX_FLOATS_IN_BUFFER, order); return fromFloatBuffer(buffer, MAX_FLOATS_IN_BUFFER, order, compression);
} }
public static CompressedFloatsIndexedSupplier fromFloatBuffer( public static CompressedFloatsIndexedSupplier fromFloatBuffer(
final FloatBuffer buffer, final int chunkFactor, final ByteOrder order final FloatBuffer buffer, final int chunkFactor, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
Preconditions.checkArgument( Preconditions.checkArgument(
@ -254,8 +290,9 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
}; };
} }
}, },
CompressedFloatBufferObjectStrategy.getBufferForOrder(order) CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, chunkFactor)
) ),
compression
); );
} }

View File

@ -36,27 +36,29 @@ import java.nio.FloatBuffer;
public class CompressedFloatsSupplierSerializer public class CompressedFloatsSupplierSerializer
{ {
public static CompressedFloatsSupplierSerializer create( public static CompressedFloatsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final ByteOrder order IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException ) throws IOException
{ {
return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order); return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order, compression);
} }
public static CompressedFloatsSupplierSerializer create( public static CompressedFloatsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException ) throws IOException
{ {
final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer( final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer(
sizePer, sizePer,
new GenericIndexedWriter<ResourceHolder<FloatBuffer>>( new GenericIndexedWriter<ResourceHolder<FloatBuffer>>(
ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order) ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)
) ),
compression
); );
return retVal; return retVal;
} }
private final int sizePer; private final int sizePer;
private final GenericIndexedWriter<ResourceHolder<FloatBuffer>> flattener; private final GenericIndexedWriter<ResourceHolder<FloatBuffer>> flattener;
private final CompressedObjectStrategy.CompressionStrategy compression;
private int numInserted = 0; private int numInserted = 0;
@ -64,11 +66,13 @@ public class CompressedFloatsSupplierSerializer
public CompressedFloatsSupplierSerializer( public CompressedFloatsSupplierSerializer(
int sizePer, int sizePer,
GenericIndexedWriter<ResourceHolder<FloatBuffer>> flattener GenericIndexedWriter<ResourceHolder<FloatBuffer>> flattener,
CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
this.sizePer = sizePer; this.sizePer = sizePer;
this.flattener = flattener; this.flattener = flattener;
this.compression = compression;
endBuffer = FloatBuffer.allocate(sizePer); endBuffer = FloatBuffer.allocate(sizePer);
endBuffer.mark(); endBuffer.mark();
@ -110,6 +114,7 @@ public class CompressedFloatsSupplierSerializer
out.write(CompressedFloatsIndexedSupplier.version); out.write(CompressedFloatsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(numInserted));
out.write(Ints.toByteArray(sizePer)); out.write(Ints.toByteArray(sizePer));
out.write(new byte[]{compression.getId()});
ByteStreams.copy(flattener.combineStreams(), out); ByteStreams.copy(flattener.combineStreams(), out);
} }
} }

View File

@ -28,14 +28,14 @@ import java.nio.LongBuffer;
/** /**
*/ */
public class CompressedLongBufferObjectStrategy extends CompressedObjectStrategy<LongBuffer> public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<LongBuffer>
{ {
public static CompressedLongBufferObjectStrategy getBufferForOrder(ByteOrder order) public static CompressedLongBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{ {
return new CompressedLongBufferObjectStrategy(order); return new CompressedLongBufferObjectStrategy(order, compression, sizePer);
} }
private CompressedLongBufferObjectStrategy(final ByteOrder order) private CompressedLongBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{ {
super( super(
order, order,
@ -64,8 +64,9 @@ public class CompressedLongBufferObjectStrategy extends CompressedObjectStrategy
{ {
return into.asLongBuffer().put(from); return into.asLongBuffer().put(from);
} }
} },
compression,
sizePer
); );
} }
} }

View File

@ -40,21 +40,25 @@ import java.util.Iterator;
*/ */
public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs> public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
{ {
public static final byte version = 0x1; public static final byte LZF_VERSION = 0x1;
public static final byte version = 0x2;
private final int totalSize; private final int totalSize;
private final int sizePer; private final int sizePer;
private final GenericIndexed<ResourceHolder<LongBuffer>> baseLongBuffers; private final GenericIndexed<ResourceHolder<LongBuffer>> baseLongBuffers;
private final CompressedObjectStrategy.CompressionStrategy compression;
CompressedLongsIndexedSupplier( CompressedLongsIndexedSupplier(
int totalSize, int totalSize,
int sizePer, int sizePer,
GenericIndexed<ResourceHolder<LongBuffer>> baseLongBuffers GenericIndexed<ResourceHolder<LongBuffer>> baseLongBuffers,
CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
this.totalSize = totalSize; this.totalSize = totalSize;
this.sizePer = sizePer; this.sizePer = sizePer;
this.baseLongBuffers = baseLongBuffers; this.baseLongBuffers = baseLongBuffers;
this.compression = compression;
} }
public int size() public int size()
@ -162,7 +166,7 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
public long getSerializedSize() public long getSerializedSize()
{ {
return baseLongBuffers.getSerializedSize() + 1 + 4 + 4; return baseLongBuffers.getSerializedSize() + 1 + 4 + 4 + 1;
} }
public void writeToChannel(WritableByteChannel channel) throws IOException public void writeToChannel(WritableByteChannel channel) throws IOException
@ -170,15 +174,17 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
channel.write(ByteBuffer.wrap(new byte[]{version})); channel.write(ByteBuffer.wrap(new byte[]{version}));
channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize)));
channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer)));
channel.write(ByteBuffer.wrap(new byte[]{compression.getId()}));
baseLongBuffers.writeToChannel(channel); baseLongBuffers.writeToChannel(channel);
} }
public CompressedLongsIndexedSupplier convertByteOrder(ByteOrder order) public CompressedLongsIndexedSupplier convertByteOrder(ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression)
{ {
return new CompressedLongsIndexedSupplier( return new CompressedLongsIndexedSupplier(
totalSize, totalSize,
sizePer, sizePer,
GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
compression
); );
} }
@ -196,23 +202,37 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
byte versionFromBuffer = buffer.get(); byte versionFromBuffer = buffer.get();
if (versionFromBuffer == version) { if (versionFromBuffer == version) {
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get());
return new CompressedLongsIndexedSupplier( return new CompressedLongsIndexedSupplier(
buffer.getInt(), totalSize,
buffer.getInt(), sizePer,
GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order)) GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
compression
);
} else if (versionFromBuffer == LZF_VERSION) {
final int totalSize = buffer.getInt();
final int sizePer = buffer.getInt();
final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF;
return new CompressedLongsIndexedSupplier(
totalSize,
sizePer,
GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)),
compression
); );
} }
throw new IAE("Unknown version[%s]", versionFromBuffer); throw new IAE("Unknown version[%s]", versionFromBuffer);
} }
public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder) public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression)
{ {
return fromLongBuffer(buffer, 0xFFFF / Longs.BYTES, byteOrder); return fromLongBuffer(buffer, 0xFFFF / Longs.BYTES, byteOrder, compression);
} }
public static CompressedLongsIndexedSupplier fromLongBuffer( public static CompressedLongsIndexedSupplier fromLongBuffer(
final LongBuffer buffer, final int chunkFactor, final ByteOrder byteOrder final LongBuffer buffer, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
Preconditions.checkArgument( Preconditions.checkArgument(
@ -259,8 +279,9 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
}; };
} }
}, },
CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder) CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor)
) ),
compression
); );
} }

View File

@ -37,20 +37,23 @@ import java.nio.LongBuffer;
public class CompressedLongsSupplierSerializer public class CompressedLongsSupplierSerializer
{ {
public static CompressedLongsSupplierSerializer create( public static CompressedLongsSupplierSerializer create(
IOPeon ioPeon, final String filenameBase, final ByteOrder order IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression
) throws IOException ) throws IOException
{ {
final int sizePer = 0xFFFF / Longs.BYTES;
final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer( final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer(
0xFFFF / Longs.BYTES, sizePer,
new GenericIndexedWriter<ResourceHolder<LongBuffer>>( new GenericIndexedWriter<ResourceHolder<LongBuffer>>(
ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order) ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)
) ),
compression
); );
return retVal; return retVal;
} }
private final int sizePer; private final int sizePer;
private final GenericIndexedWriter<ResourceHolder<LongBuffer>> flattener; private final GenericIndexedWriter<ResourceHolder<LongBuffer>> flattener;
private final CompressedObjectStrategy.CompressionStrategy compression;
private int numInserted = 0; private int numInserted = 0;
@ -58,11 +61,13 @@ public class CompressedLongsSupplierSerializer
public CompressedLongsSupplierSerializer( public CompressedLongsSupplierSerializer(
int sizePer, int sizePer,
GenericIndexedWriter<ResourceHolder<LongBuffer>> flattener GenericIndexedWriter<ResourceHolder<LongBuffer>> flattener,
CompressedObjectStrategy.CompressionStrategy compression
) )
{ {
this.sizePer = sizePer; this.sizePer = sizePer;
this.flattener = flattener; this.flattener = flattener;
this.compression = compression;
endBuffer = LongBuffer.allocate(sizePer); endBuffer = LongBuffer.allocate(sizePer);
endBuffer.mark(); endBuffer.mark();
@ -104,6 +109,7 @@ public class CompressedLongsSupplierSerializer
out.write(CompressedLongsIndexedSupplier.version); out.write(CompressedLongsIndexedSupplier.version);
out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(numInserted));
out.write(Ints.toByteArray(sizePer)); out.write(Ints.toByteArray(sizePer));
out.write(new byte[]{compression.getId()});
ByteStreams.copy(flattener.combineStreams(), out); ByteStreams.copy(flattener.combineStreams(), out);
} }
} }

View File

@ -21,31 +21,218 @@ package io.druid.segment.data;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.CloseQuietly;
import com.google.common.collect.Maps;
import com.ning.compress.lzf.ChunkEncoder; import com.ning.compress.lzf.ChunkEncoder;
import com.ning.compress.lzf.LZFChunk; import com.ning.compress.lzf.LZFChunk;
import com.ning.compress.lzf.LZFDecoder; import com.ning.compress.lzf.LZFDecoder;
import io.druid.collections.ResourceHolder; import io.druid.collections.ResourceHolder;
import io.druid.segment.CompressedPools; import io.druid.segment.CompressedPools;
import net.jpountz.lz4.LZ4Factory;
import net.jpountz.lz4.LZ4FastDecompressor;
import net.jpountz.lz4.LZ4SafeDecompressor;
import net.jpountz.lz4.LZ4UnknownSizeDecompressor;
import java.io.IOException; import java.io.IOException;
import java.nio.Buffer; import java.nio.Buffer;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.util.Map;
/** /**
*/ */
public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrategy<ResourceHolder<T>> public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrategy<ResourceHolder<T>>
{ {
private final ByteOrder order; public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = CompressionStrategy.LZ4;
private final BufferConverter<T> converter;
public static enum CompressionStrategy {
LZF ((byte)0x0)
{
@Override
public Decompressor getDecompressor()
{
return new LZFDecompressor();
}
@Override
public Compressor getCompressor()
{
return new LZFCompressor();
}
},
LZ4 ((byte)0x1) {
@Override
public Decompressor getDecompressor()
{
return new LZ4Decompressor();
}
@Override
public Compressor getCompressor()
{
return new LZ4Compressor();
}
};
final byte id;
CompressionStrategy(byte id) {
this.id = id;
}
public byte getId()
{
return id;
}
public abstract Compressor getCompressor();
public abstract Decompressor getDecompressor();
static final Map<Byte, CompressionStrategy> idMap = Maps.newHashMap();
static {
for(CompressionStrategy strategy : CompressionStrategy.values()) idMap.put(strategy.getId(), strategy);
}
public static CompressionStrategy forId(byte id)
{
return idMap.get(id);
}
}
public static interface Decompressor
{
/**
* Implementations of this method are expected to call out.flip() after writing to the output buffer
*
* @param in
* @param numBytes
* @param out
*/
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out);
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize);
}
public static interface Compressor
{
/**
* Currently assumes buf is an array backed ByteBuffer
*
* @param bytes
* @return
*/
public byte[] compress(byte[] bytes);
}
public static class LZFDecompressor implements Decompressor
{
@Override
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out)
{
final byte[] bytes = new byte[numBytes];
in.get(bytes);
try (final ResourceHolder<byte[]> outputBytesHolder = CompressedPools.getOutputBytes()) {
final byte[] outputBytes = outputBytesHolder.get();
final int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes);
out.put(outputBytes, 0, numDecompressedBytes);
out.flip();
}
catch (IOException e) {
Throwables.propagate(e);
}
}
@Override
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize)
{
decompress(in, numBytes, out);
}
}
public static class LZFCompressor implements Compressor
{
@Override
public byte[] compress(byte[] bytes)
{
final ResourceHolder<ChunkEncoder> encoder = CompressedPools.getChunkEncoder();
LZFChunk chunk = encoder.get().encodeChunk(bytes, 0, bytes.length);
CloseQuietly.close(encoder);
return chunk.getData();
}
}
public static class LZ4Decompressor implements Decompressor
{
private final LZ4SafeDecompressor lz4 = LZ4Factory.fastestJavaInstance().safeDecompressor();
private final LZ4FastDecompressor lz4Fast = LZ4Factory.fastestJavaInstance().fastDecompressor();
@Override
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out)
{
final byte[] bytes = new byte[numBytes];
in.get(bytes);
try (final ResourceHolder<byte[]> outputBytesHolder = CompressedPools.getOutputBytes()) {
final byte[] outputBytes = outputBytesHolder.get();
final int numDecompressedBytes = lz4.decompress(bytes, 0, bytes.length, outputBytes, 0, outputBytes.length);
out.put(outputBytes, 0, numDecompressedBytes);
out.flip();
}
catch (IOException e) {
Throwables.propagate(e);
}
}
@Override
public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize)
{
final byte[] bytes = new byte[numBytes];
in.get(bytes);
try (final ResourceHolder<byte[]> outputBytesHolder = CompressedPools.getOutputBytes()) {
final byte[] outputBytes = outputBytesHolder.get();
lz4Fast.decompress(bytes, 0, outputBytes, 0, decompressedSize);
out.put(outputBytes, 0, decompressedSize);
out.flip();
}
catch (IOException e) {
Throwables.propagate(e);
}
}
}
public static class LZ4Compressor implements Compressor
{
private final net.jpountz.lz4.LZ4Compressor lz4 = LZ4Factory.fastestJavaInstance().highCompressor();
@Override
public byte[] compress(byte[] bytes)
{
final byte[] intermediate = new byte[lz4.maxCompressedLength(bytes.length)];
final int outputBytes = lz4.compress(bytes, 0, bytes.length, intermediate, 0, intermediate.length);
final byte[] out = new byte[outputBytes];
System.arraycopy(intermediate, 0, out, 0, outputBytes);
return out;
}
}
protected final ByteOrder order;
protected final BufferConverter<T> converter;
protected final Decompressor decompressor;
private final Compressor compressor;
protected CompressedObjectStrategy( protected CompressedObjectStrategy(
final ByteOrder order, final ByteOrder order,
final BufferConverter<T> converter final BufferConverter<T> converter,
final CompressionStrategy compression
) )
{ {
this.order = order; this.order = order;
this.converter = converter; this.converter = converter;
this.decompressor = compression.getDecompressor();
this.compressor = compression.getCompressor();
} }
@Override @Override
@ -58,24 +245,12 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
@Override @Override
public ResourceHolder<T> fromByteBuffer(ByteBuffer buffer, int numBytes) public ResourceHolder<T> fromByteBuffer(ByteBuffer buffer, int numBytes)
{ {
byte[] bytes = new byte[numBytes];
buffer.get(bytes);
final ResourceHolder<ByteBuffer> bufHolder = CompressedPools.getByteBuf(order); final ResourceHolder<ByteBuffer> bufHolder = CompressedPools.getByteBuf(order);
final ByteBuffer buf = bufHolder.get(); final ByteBuffer buf = bufHolder.get();
buf.position(0); buf.position(0);
buf.limit(buf.capacity()); buf.limit(buf.capacity());
try { decompress(buffer, numBytes, buf);
final ResourceHolder<byte[]> outputBytesHolder = CompressedPools.getOutputBytes();
byte[] outputBytes = outputBytesHolder.get();
int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes);
buf.put(outputBytes, 0, numDecompressedBytes);
buf.flip();
CloseQuietly.close(outputBytesHolder);
return new ResourceHolder<T>() return new ResourceHolder<T>()
{ {
@Override @Override
@ -91,23 +266,28 @@ public class CompressedObjectStrategy<T extends Buffer> implements ObjectStrateg
} }
}; };
} }
catch (IOException e) {
throw Throwables.propagate(e); protected void decompress(
} ByteBuffer buffer,
int numBytes,
ByteBuffer buf
)
{
decompressor.decompress(buffer, numBytes, buf);
} }
@Override @Override
public byte[] toBytes(ResourceHolder<T> holder) public byte[] toBytes(ResourceHolder<T> holder)
{ {
T val = holder.get(); T val = holder.get();
ByteBuffer buf = ByteBuffer.allocate(converter.sizeOf(val.remaining())).order(order); ByteBuffer buf = bufferFor(val);
converter.combine(buf, val); converter.combine(buf, val);
return compressor.compress(buf.array());
}
final ResourceHolder<ChunkEncoder> encoder = CompressedPools.getChunkEncoder(); protected ByteBuffer bufferFor(T val)
LZFChunk chunk = encoder.get().encodeChunk(buf.array(), 0, buf.array().length); {
CloseQuietly.close(encoder); return ByteBuffer.allocate(converter.sizeOf(val.remaining())).order(order);
return chunk.getData();
} }
@Override @Override

View File

@ -0,0 +1,56 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment.data;
import java.nio.Buffer;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public abstract class FixedSizeCompressedObjectStrategy<T extends Buffer> extends CompressedObjectStrategy<T>
{
private final int sizePer;
protected FixedSizeCompressedObjectStrategy(
ByteOrder order,
BufferConverter<T> converter,
CompressionStrategy compression,
int sizePer
)
{
super(order, converter, compression);
this.sizePer = sizePer;
}
public int getSize() {
return sizePer;
}
@Override
protected ByteBuffer bufferFor(T val)
{
return ByteBuffer.allocate(converter.sizeOf(getSize())).order(order);
}
@Override
protected void decompress(ByteBuffer buffer, int numBytes, ByteBuffer buf)
{
decompressor.decompress(buffer, numBytes, buf, converter.sizeOf(getSize()));
}
}

View File

@ -37,6 +37,7 @@ import java.util.List;
*/ */
public class InMemoryCompressedFloats implements IndexedFloats public class InMemoryCompressedFloats implements IndexedFloats
{ {
public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY;
private final CompressedFloatBufferObjectStrategy strategy; private final CompressedFloatBufferObjectStrategy strategy;
private final int sizePer; private final int sizePer;
@ -56,7 +57,11 @@ public class InMemoryCompressedFloats implements IndexedFloats
) )
{ {
this.sizePer = sizePer; this.sizePer = sizePer;
strategy = CompressedFloatBufferObjectStrategy.getBufferForOrder(order); strategy = CompressedFloatBufferObjectStrategy.getBufferForOrder(
order,
COMPRESSION,
sizePer
);
endBuffer = FloatBuffer.allocate(sizePer); endBuffer = FloatBuffer.allocate(sizePer);
endBuffer.mark(); endBuffer.mark();
@ -184,7 +189,8 @@ public class InMemoryCompressedFloats implements IndexedFloats
Arrays.<ResourceHolder<FloatBuffer>>asList(StupidResourceHolder.create(endBufCopy)) Arrays.<ResourceHolder<FloatBuffer>>asList(StupidResourceHolder.create(endBufCopy))
), ),
strategy strategy
) ),
COMPRESSION
); );
} }

View File

@ -38,6 +38,7 @@ import java.util.List;
*/ */
public class InMemoryCompressedLongs implements IndexedLongs public class InMemoryCompressedLongs implements IndexedLongs
{ {
public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY;
private final CompressedLongBufferObjectStrategy strategy; private final CompressedLongBufferObjectStrategy strategy;
private final int sizePer; private final int sizePer;
@ -57,7 +58,11 @@ public class InMemoryCompressedLongs implements IndexedLongs
) )
{ {
this.sizePer = sizePer; this.sizePer = sizePer;
strategy = CompressedLongBufferObjectStrategy.getBufferForOrder(order); strategy = CompressedLongBufferObjectStrategy.getBufferForOrder(
order,
COMPRESSION,
sizePer
);
endBuffer = LongBuffer.allocate(sizePer); endBuffer = LongBuffer.allocate(sizePer);
endBuffer.mark(); endBuffer.mark();
@ -195,7 +200,8 @@ public class InMemoryCompressedLongs implements IndexedLongs
Arrays.<ResourceHolder<LongBuffer>>asList(StupidResourceHolder.create(longBufCopy)) Arrays.<ResourceHolder<LongBuffer>>asList(StupidResourceHolder.create(longBufCopy))
), ),
strategy strategy
) ),
COMPRESSION
); );
} }

View File

@ -35,6 +35,8 @@ import org.easymock.IAnswer;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
@ -100,13 +102,14 @@ public class ChainedExecutionQueryRunnerTest
runner3 runner3
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
final Sequence seq = chainedRunner.run( final Sequence seq = chainedRunner.run(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource("test") .dataSource("test")
.intervals("2014/2015") .intervals("2014/2015")
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count"))) .aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
.build() .build(),
context
); );
Future resultFuture = Executors.newFixedThreadPool(1).submit( Future resultFuture = Executors.newFixedThreadPool(1).submit(
@ -207,14 +210,15 @@ public class ChainedExecutionQueryRunnerTest
runner3 runner3
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
final Sequence seq = chainedRunner.run( final Sequence seq = chainedRunner.run(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource("test") .dataSource("test")
.intervals("2014/2015") .intervals("2014/2015")
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count"))) .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() .build(),
context
); );
Future resultFuture = Executors.newFixedThreadPool(1).submit( Future resultFuture = Executors.newFixedThreadPool(1).submit(
@ -275,7 +279,7 @@ public class ChainedExecutionQueryRunnerTest
} }
@Override @Override
public Sequence<Integer> run(Query<Integer> query) public Sequence<Integer> run(Query<Integer> query, Map<String, Object> context)
{ {
hasStarted = true; hasStarted = true;
start.countDown(); start.countDown();

View File

@ -0,0 +1,266 @@
package io.druid.query;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Maps;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.segment.SegmentMissingException;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
public class RetryQueryRunnerTest
{
private final ObjectMapper jsonMapper = new DefaultObjectMapper();
final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory(
"idx",
"index"
),
QueryRunnerTestHelper.qualityUniques
)
)
.build();
@Test
public void testRunWithMissingSegments() throws Exception
{
Map<String, Object> context = new MapMaker().makeMap();
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(Query query, Map context)
{
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add(
new SegmentDescriptor(
new Interval(
178888,
1999999
), "test", 1
)
);
return Sequences.empty();
}
},
new RetryQueryRunnerConfig()
{
private int numTries = 0;
private boolean returnPartialResults = true;
public int numTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
},
jsonMapper
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Assert.assertTrue(
"Should have one entry in the list of missing segments",
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 1
);
Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0);
}
@Test
public void testRetry() throws Exception
{
Map<String, Object> context = new MapMaker().makeMap();
context.put("count", 0);
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context
)
{
if ((int) context.get("count") == 0) {
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add(
new SegmentDescriptor(
new Interval(
178888,
1999999
), "test", 1
)
);
context.put("count", 1);
return Sequences.empty();
} else {
return Sequences.simple(
Arrays.asList(
new Result<>(
new DateTime(),
new TimeseriesResultValue(
Maps.<String, Object>newHashMap()
)
)
)
);
}
}
},
new RetryQueryRunnerConfig()
{
private int numTries = 1;
private boolean returnPartialResults = true;
public int numTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
},
jsonMapper
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1);
Assert.assertTrue(
"Should have nothing in missingSegment list",
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 0
);
}
@Test
public void testRetryMultiple() throws Exception
{
Map<String, Object> context = new MapMaker().makeMap();
context.put("count", 0);
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context
)
{
if ((int) context.get("count") < 3) {
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add(
new SegmentDescriptor(
new Interval(
178888,
1999999
), "test", 1
)
);
context.put("count", (int) context.get("count") + 1);
return Sequences.empty();
} else {
return Sequences.simple(
Arrays.asList(
new Result<>(
new DateTime(),
new TimeseriesResultValue(
Maps.<String, Object>newHashMap()
)
)
)
);
}
}
},
new RetryQueryRunnerConfig()
{
private int numTries = 4;
private boolean returnPartialResults = true;
public int numTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
},
jsonMapper
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1);
Assert.assertTrue(
"Should have nothing in missingSegment list",
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 0
);
}
@Test(expected = SegmentMissingException.class)
public void testException() throws Exception
{
Map<String, Object> context = new MapMaker().makeMap();
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
RetryQueryRunner<Result<TimeseriesResultValue>> runner = new RetryQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context
)
{
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).add(
new SegmentDescriptor(
new Interval(
178888,
1999999
), "test", 1
)
);
return Sequences.empty();
}
},
new RetryQueryRunnerConfig()
{
private int numTries = 1;
private boolean returnPartialResults = false;
public int numTries() { return numTries; }
public boolean returnPartialResults() { return returnPartialResults; }
},
jsonMapper
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Assert.assertTrue(
"Should have one entry in the list of missing segments",
((List) context.get(RetryQueryRunner.MISSING_SEGMENTS_KEY)).size() == 1
);
}
}

View File

@ -22,6 +22,7 @@ package io.druid.query;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
@ -35,9 +36,13 @@ import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map;
public class TimewarpOperatorTest public class TimewarpOperatorTest
{ {
public static final ImmutableMap<String, Object> CONTEXT = ImmutableMap.of();
TimewarpOperator<Result<TimeseriesResultValue>> testOperator = new TimewarpOperator<>( TimewarpOperator<Result<TimeseriesResultValue>> testOperator = new TimewarpOperator<>(
new Interval(new DateTime("2014-01-01"), new DateTime("2014-01-15")), new Interval(new DateTime("2014-01-01"), new DateTime("2014-01-15")),
new Period("P1W"), new Period("P1W"),
@ -75,7 +80,10 @@ public class TimewarpOperatorTest
new QueryRunner<Result<TimeseriesResultValue>>() new QueryRunner<Result<TimeseriesResultValue>>()
{ {
@Override @Override
public Sequence<Result<TimeseriesResultValue>> run(Query<Result<TimeseriesResultValue>> query) public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context
)
{ {
return Sequences.simple( return Sequences.simple(
ImmutableList.of( ImmutableList.of(
@ -120,7 +128,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 5)) new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 5))
) )
), ),
Sequences.toList(queryRunner.run(query), Lists.<Result<TimeseriesResultValue>>newArrayList()) Sequences.toList(queryRunner.run(query, CONTEXT), Lists.<Result<TimeseriesResultValue>>newArrayList())
); );
@ -134,13 +142,21 @@ public class TimewarpOperatorTest
new QueryRunner<Result<TimeBoundaryResultValue>>() new QueryRunner<Result<TimeBoundaryResultValue>>()
{ {
@Override @Override
public Sequence<Result<TimeBoundaryResultValue>> run(Query<Result<TimeBoundaryResultValue>> query) public Sequence<Result<TimeBoundaryResultValue>> run(
Query<Result<TimeBoundaryResultValue>> query,
Map<String, Object> context
)
{ {
return Sequences.simple( return Sequences.simple(
ImmutableList.of( ImmutableList.of(
new Result<>( new Result<>(
new DateTime("2014-01-12"), new DateTime("2014-01-12"),
new TimeBoundaryResultValue(ImmutableMap.<String, Object>of("maxTime", new DateTime("2014-01-12"))) new TimeBoundaryResultValue(
ImmutableMap.<String, Object>of(
"maxTime",
new DateTime("2014-01-12")
)
)
) )
) )
); );
@ -161,7 +177,10 @@ public class TimewarpOperatorTest
new TimeBoundaryResultValue(ImmutableMap.<String, Object>of("maxTime", new DateTime("2014-08-02"))) new TimeBoundaryResultValue(ImmutableMap.<String, Object>of("maxTime", new DateTime("2014-08-02")))
) )
), ),
Sequences.toList(timeBoundaryRunner.run(timeBoundaryQuery), Lists.<Result<TimeBoundaryResultValue>>newArrayList()) Sequences.toList(
timeBoundaryRunner.run(timeBoundaryQuery, CONTEXT),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
)
); );
} }
@ -173,7 +192,10 @@ public class TimewarpOperatorTest
new QueryRunner<Result<TimeseriesResultValue>>() new QueryRunner<Result<TimeseriesResultValue>>()
{ {
@Override @Override
public Sequence<Result<TimeseriesResultValue>> run(Query<Result<TimeseriesResultValue>> query) public Sequence<Result<TimeseriesResultValue>> run(
Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context
)
{ {
return Sequences.simple( return Sequences.simple(
ImmutableList.of( ImmutableList.of(
@ -210,7 +232,7 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 3)) new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 3))
) )
), ),
Sequences.toList(queryRunner.run(query), Lists.<Result<TimeseriesResultValue>>newArrayList()) Sequences.toList(queryRunner.run(query, Maps.<String, Object>newHashMap()), Lists.<Result<TimeseriesResultValue>>newArrayList())
); );
} }
} }

View File

@ -82,6 +82,7 @@ import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -411,7 +412,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>() new QueryRunner<Row>()
{ {
@Override @Override
public Sequence run(Query<Row> query) public Sequence run(Query<Row> query, Map<String, Object> context)
{ {
// simulate two daily segments // simulate two daily segments
final Query query1 = query.withQuerySegmentSpec( final Query query1 = query.withQuerySegmentSpec(
@ -420,7 +421,7 @@ public class GroupByQueryRunnerTest
final Query query2 = query.withQuerySegmentSpec( final Query query2 = query.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
); );
return Sequences.concat(runner.run(query1), runner.run(query2)); return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
} }
} }
); );
@ -437,8 +438,9 @@ public class GroupByQueryRunnerTest
createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
); );
TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery), "direct"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, context), "direct");
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
List<Row> allGranExpectedResults = Arrays.asList( List<Row> allGranExpectedResults = Arrays.asList(
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L),
@ -452,8 +454,8 @@ public class GroupByQueryRunnerTest
createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
); );
TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery), "direct"); TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, context), "direct");
TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery), "merged"); TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged");
} }
@Test @Test
@ -495,9 +497,9 @@ public class GroupByQueryRunnerTest
); );
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner); QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects( TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery), String.format("limit: %d", limit) Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), String.format("limit: %d", limit)
); );
} }
@ -603,7 +605,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>() new QueryRunner<Row>()
{ {
@Override @Override
public Sequence run(Query<Row> query) public Sequence run(Query<Row> query, Map<String, Object> context)
{ {
// simulate two daily segments // simulate two daily segments
final Query query1 = query.withQuerySegmentSpec( final Query query1 = query.withQuerySegmentSpec(
@ -612,12 +614,12 @@ public class GroupByQueryRunnerTest
final Query query2 = query.withQuerySegmentSpec( final Query query2 = query.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
); );
return Sequences.concat(runner.run(query1), runner.run(query2)); return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
} }
} }
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
} }
@Test @Test
@ -653,10 +655,11 @@ public class GroupByQueryRunnerTest
); );
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner); QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects( TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
); );
} }
@ -693,9 +696,10 @@ public class GroupByQueryRunnerTest
); );
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner); QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects( TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
); );
} }
@ -732,9 +736,10 @@ public class GroupByQueryRunnerTest
); );
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner); QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects( TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
); );
} }
@ -774,7 +779,7 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>() new QueryRunner<Row>()
{ {
@Override @Override
public Sequence run(Query<Row> query) public Sequence run(Query<Row> query, Map<String, Object> context)
{ {
// simulate two daily segments // simulate two daily segments
final Query query1 = query.withQuerySegmentSpec( final Query query1 = query.withQuerySegmentSpec(
@ -783,12 +788,13 @@ public class GroupByQueryRunnerTest
final Query query2 = query.withQuerySegmentSpec( final Query query2 = query.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
); );
return Sequences.concat(runner.run(query1), runner.run(query2)); return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
} }
} }
); );
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
} }
@Test @Test
@ -827,12 +833,13 @@ public class GroupByQueryRunnerTest
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest( QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(
configSupplier, configSupplier,
new DefaultObjectMapper(), new DefaultObjectMapper(),
engine engine
).mergeResults(runner); ).mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
} }
@Test @Test
@ -864,8 +871,8 @@ public class GroupByQueryRunnerTest
createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L), createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L),
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
final GroupByQueryEngine engine = new GroupByQueryEngine( final GroupByQueryEngine engine = new GroupByQueryEngine(
configSupplier, configSupplier,
new StupidPool<ByteBuffer>( new StupidPool<ByteBuffer>(
@ -885,7 +892,7 @@ public class GroupByQueryRunnerTest
new DefaultObjectMapper(), new DefaultObjectMapper(),
engine engine
).mergeResults(runner); ).mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
} }
@Test @Test
@ -918,7 +925,8 @@ public class GroupByQueryRunnerTest
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L) createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
); );
TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
final GroupByQueryEngine engine = new GroupByQueryEngine( final GroupByQueryEngine engine = new GroupByQueryEngine(
configSupplier, configSupplier,
new StupidPool<ByteBuffer>( new StupidPool<ByteBuffer>(
@ -938,7 +946,7 @@ public class GroupByQueryRunnerTest
new DefaultObjectMapper(), new DefaultObjectMapper(),
engine engine
).mergeResults(runner); ).mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
} }
// A subquery identical to the query should yield identical results // A subquery identical to the query should yield identical results
@ -1452,7 +1460,8 @@ public class GroupByQueryRunnerTest
toolChest toolChest
); );
Sequence<Row> queryResult = theRunner.run(query); HashMap<String,Object> context = new HashMap<String, Object>();
Sequence<Row> queryResult = theRunner.run(query, context);
return Sequences.toList(queryResult, Lists.<Row>newArrayList()); return Sequences.toList(queryResult, Lists.<Row>newArrayList());
} }

View File

@ -43,6 +43,7 @@ import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Map;
/** /**
*/ */
@ -92,7 +93,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
QueryRunner timeseriesRunner = new QueryRunner() QueryRunner timeseriesRunner = new QueryRunner()
{ {
@Override @Override
public Sequence run(Query query) public Sequence run(Query query, Map metadata)
{ {
TimeseriesQuery tsQuery = (TimeseriesQuery) query; TimeseriesQuery tsQuery = (TimeseriesQuery) query;
@ -105,7 +106,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
.setDimFilter(tsQuery.getDimensionsFilter()) .setDimFilter(tsQuery.getDimensionsFilter())
.setAggregatorSpecs(tsQuery.getAggregatorSpecs()) .setAggregatorSpecs(tsQuery.getAggregatorSpecs())
.setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs())
.build() .build(),
metadata
), ),
new Function<Row, Result<TimeseriesResultValue>>() new Function<Row, Result<TimeseriesResultValue>>()
{ {

View File

@ -38,6 +38,7 @@ import io.druid.segment.column.ValueType;
import junit.framework.Assert; import junit.framework.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -106,6 +107,7 @@ public class SegmentAnalyzerTest
final SegmentMetadataQuery query = new SegmentMetadataQuery( final SegmentMetadataQuery query = new SegmentMetadataQuery(
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null
); );
return Sequences.toList(query.run(runner), Lists.<SegmentAnalysis>newArrayList()); HashMap<String,Object> context = new HashMap<String, Object>();
return Sequences.toList(query.run(runner, context), Lists.<SegmentAnalysis>newArrayList());
} }
} }

View File

@ -41,6 +41,8 @@ import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
public class SegmentMetadataQueryTest public class SegmentMetadataQueryTest
{ {
@ -73,9 +75,9 @@ public class SegmentMetadataQueryTest
.toInclude(new ListColumnIncluderator(Arrays.asList("placement"))) .toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.merge(true) .merge(true)
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<SegmentAnalysis> results = Sequences.toList( Iterable<SegmentAnalysis> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<SegmentAnalysis>newArrayList() Lists.<SegmentAnalysis>newArrayList()
); );
SegmentAnalysis val = results.iterator().next(); SegmentAnalysis val = results.iterator().next();

View File

@ -47,6 +47,7 @@ import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -371,8 +372,9 @@ public class SearchQueryRunnerTest
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults) private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
{ {
HashMap<String,List> context = new HashMap<String, List>();
Iterable<Result<SearchResultValue>> results = Sequences.toList( Iterable<Result<SearchResultValue>> results = Sequences.toList(
runner.run(searchQuery), runner.run(searchQuery, context),
Lists.<Result<SearchResultValue>>newArrayList() Lists.<Result<SearchResultValue>>newArrayList()
); );

View File

@ -47,6 +47,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -92,9 +93,9 @@ public class SelectQueryRunnerTest
new PagingSpec(null, 3), new PagingSpec(null, 3),
null null
); );
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList( Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList() Lists.<Result<SelectResultValue>>newArrayList()
); );
@ -161,9 +162,9 @@ public class SelectQueryRunnerTest
new PagingSpec(null, 3), new PagingSpec(null, 3),
null null
); );
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList( Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList() Lists.<Result<SelectResultValue>>newArrayList()
); );
@ -221,9 +222,9 @@ public class SelectQueryRunnerTest
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
null null
); );
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList( Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList() Lists.<Result<SelectResultValue>>newArrayList()
); );
@ -281,9 +282,9 @@ public class SelectQueryRunnerTest
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3), new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
null null
); );
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList( Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList() Lists.<Result<SelectResultValue>>newArrayList()
); );

View File

@ -21,11 +21,13 @@ package io.druid.query.timeboundary;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.query.Druids; import io.druid.query.Druids;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result; import io.druid.query.Result;
import io.druid.query.RetryQueryRunner;
import io.druid.query.TableDataSource; import io.druid.query.TableDataSource;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.junit.Assert; import org.junit.Assert;
@ -36,7 +38,9 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
*/ */
@ -67,9 +71,9 @@ public class TimeBoundaryQueryRunnerTest
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
.dataSource("testing") .dataSource("testing")
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList( Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery), runner.run(timeBoundaryQuery, context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList() Lists.<Result<TimeBoundaryResultValue>>newArrayList()
); );
TimeBoundaryResultValue val = results.iterator().next().getValue(); TimeBoundaryResultValue val = results.iterator().next().getValue();
@ -88,9 +92,10 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing") .dataSource("testing")
.bound(TimeBoundaryQuery.MAX_TIME) .bound(TimeBoundaryQuery.MAX_TIME)
.build(); .build();
Map<String, Object> context = new MapMaker().makeMap();
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList( Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery), runner.run(timeBoundaryQuery, context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList() Lists.<Result<TimeBoundaryResultValue>>newArrayList()
); );
TimeBoundaryResultValue val = results.iterator().next().getValue(); TimeBoundaryResultValue val = results.iterator().next().getValue();
@ -109,9 +114,10 @@ public class TimeBoundaryQueryRunnerTest
.dataSource("testing") .dataSource("testing")
.bound(TimeBoundaryQuery.MIN_TIME) .bound(TimeBoundaryQuery.MIN_TIME)
.build(); .build();
Map<String, Object> context = new MapMaker().makeMap();
context.put(RetryQueryRunner.MISSING_SEGMENTS_KEY, Lists.newArrayList());
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList( Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery), runner.run(timeBoundaryQuery, context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList() Lists.<Result<TimeBoundaryResultValue>>newArrayList()
); );
TimeBoundaryResultValue val = results.iterator().next().getValue(); TimeBoundaryResultValue val = results.iterator().next().getValue();

View File

@ -21,12 +21,19 @@ package io.druid.query.timeseries;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.query.Druids; import io.druid.query.Druids;
import io.druid.query.Query;
import io.druid.query.QueryConfig; import io.druid.query.QueryConfig;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QueryToolChest;
import io.druid.query.Result; import io.druid.query.Result;
import io.druid.query.TableDataSource;
import io.druid.query.UnionDataSource;
import io.druid.query.UnionQueryRunner;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
@ -38,11 +45,22 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class TimeSeriesUnionQueryRunnerTest public class TimeSeriesUnionQueryRunnerTest
{ {
private final QueryRunner runner;
public TimeSeriesUnionQueryRunnerTest(
QueryRunner runner
)
{
this.runner = runner;
}
@Parameterized.Parameters @Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException public static Collection<?> constructorFeeder() throws IOException
{ {
@ -55,15 +73,6 @@ public class TimeSeriesUnionQueryRunnerTest
); );
} }
private final QueryRunner runner;
public TimeSeriesUnionQueryRunnerTest(
QueryRunner runner
)
{
this.runner = runner;
}
@Test @Test
public void testUnionTimeseries() public void testUnionTimeseries()
{ {
@ -97,15 +106,155 @@ public class TimeSeriesUnionQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
} }
@Test
public void testUnionResultMerging()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(
new UnionDataSource(
Lists.newArrayList(
new TableDataSource("ds1"),
new TableDataSource("ds2")
)
)
)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
Arrays.<AggregatorFactory>asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory(
"idx",
"index"
)
)
)
.build();
QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(new QueryConfig());
QueryRunner mergingrunner = toolChest.mergeResults(
new UnionQueryRunner<Result<TimeseriesResultValue>>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override
public Sequence<Result<TimeseriesResultValue>> run(Query<Result<TimeseriesResultValue>> query,
Map<String, Object> context)
{
if (query.getDataSource().equals(new TableDataSource("ds1"))) {
return Sequences.simple(
Lists.newArrayList(
new Result<TimeseriesResultValue>(
new DateTime("2011-04-02"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
"rows",
1L,
"idx",
2L
)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-03"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
"rows",
3L,
"idx",
4L
)
)
)
)
);
} else {
return Sequences.simple(
Lists.newArrayList(
new Result<TimeseriesResultValue>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
"rows",
5L,
"idx",
6L
)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-02"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
"rows",
7L,
"idx",
8L
)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-04"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
"rows",
9L,
"idx",
10L
)
)
)
)
);
}
}
},
toolChest
)
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<TimeseriesResultValue>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 5L, "idx", 6L)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-02"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 8L, "idx", 10L)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-03"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 3L, "idx", 4L)
)
),
new Result<TimeseriesResultValue>(
new DateTime("2011-04-04"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 9L, "idx", 10L)
)
)
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
mergingrunner.run(query, Maps.<String, Object>newHashMap()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
System.out.println(results);
TestHelper.assertExpectedResults(expectedResults, results);
}
} }

View File

@ -43,6 +43,7 @@ import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.util.HashMap;
import java.util.List; import java.util.List;
public class TimeseriesQueryRunnerBonusTest public class TimeseriesQueryRunnerBonusTest
@ -110,9 +111,9 @@ public class TimeseriesQueryRunnerBonusTest
) )
) )
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
return Sequences.toList( return Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
} }

View File

@ -21,6 +21,7 @@ package io.druid.query.timeseries;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import io.druid.granularity.PeriodGranularity; import io.druid.granularity.PeriodGranularity;
@ -52,6 +53,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Map;
import java.util.List; import java.util.List;
/** /**
@ -59,6 +61,9 @@ import java.util.List;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class TimeseriesQueryRunnerTest public class TimeseriesQueryRunnerTest
{ {
public static final Map<String, Object> CONTEXT = ImmutableMap.of();
@Parameterized.Parameters @Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException public static Collection<?> constructorFeeder() throws IOException
{ {
@ -102,7 +107,7 @@ public class TimeseriesQueryRunnerTest
DateTime expectedLast = new DateTime("2011-04-15"); DateTime expectedLast = new DateTime("2011-04-15");
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
@ -167,7 +172,7 @@ public class TimeseriesQueryRunnerTest
DateTime expectedLast = new DateTime("2011-04-15"); DateTime expectedLast = new DateTime("2011-04-15");
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
Result<TimeseriesResultValue> result = results.iterator().next(); Result<TimeseriesResultValue> result = results.iterator().next();
@ -214,7 +219,7 @@ public class TimeseriesQueryRunnerTest
DateTime expectedLast = new DateTime("2011-04-15"); DateTime expectedLast = new DateTime("2011-04-15");
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
@ -280,7 +285,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
@ -327,7 +332,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
@ -369,7 +374,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1), runner.run(query1, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults1, results1); TestHelper.assertExpectedResults(expectedResults1, results1);
@ -406,7 +411,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
runner.run(query2), runner.run(query2, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults2, results2); TestHelper.assertExpectedResults(expectedResults2, results2);
@ -459,7 +464,122 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1), runner.run(query1, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults1, results1);
}
@Test
public void testTimeseriesQueryZeroFilling()
{
TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market")
.granularity(QueryGranularity.HOUR)
.intervals(
Arrays.asList(
new Interval(
"2011-04-14T00:00:00.000Z/2011-05-01T00:00:00.000Z"
)
)
)
.aggregators(
Arrays.<AggregatorFactory>asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory(
"idx",
"index"
)
)
)
.build();
List<Result<TimeseriesResultValue>> lotsOfZeroes = Lists.newArrayList();
for (final Long millis : QueryGranularity.HOUR.iterable(
new DateTime("2011-04-14T01").getMillis(),
new DateTime("2011-04-15").getMillis()
)) {
lotsOfZeroes.add(
new Result<>(
new DateTime(millis),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 0L, "idx", 0L)
)
)
);
}
List<Result<TimeseriesResultValue>> expectedResults1 = Lists.newArrayList(
Iterables.concat(
Arrays.asList(
new Result<>(
new DateTime("2011-04-14T00"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 4907L)
)
)
),
lotsOfZeroes,
Arrays.asList(
new Result<>(
new DateTime("2011-04-15T00"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 4717L)
)
)
)
)
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults1, results1);
}
@Test
public void testTimeseriesQueryGranularityNotAlignedWithRollupGranularity()
{
TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market")
.granularity(
new PeriodGranularity(
new Period("PT1H"),
new DateTime(60000),
DateTimeZone.UTC
)
)
.intervals(
Arrays.asList(
new Interval(
"2011-04-15T00:00:00.000Z/2012"
)
)
)
.aggregators(
Arrays.<AggregatorFactory>asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory(
"idx",
"index"
)
)
)
.build();
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
new Result<TimeseriesResultValue>(
new DateTime("2011-04-14T23:01Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("rows", 13L, "idx", 4717L)
)
)
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults1, results1); TestHelper.assertExpectedResults(expectedResults1, results1);
@ -499,9 +619,8 @@ public class TimeseriesQueryRunnerTest
) )
) )
); );
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1), runner.run(query1, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults1, results1); TestHelper.assertExpectedResults(expectedResults1, results1);
@ -539,7 +658,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
runner.run(query2), runner.run(query2, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults2, results2); TestHelper.assertExpectedResults(expectedResults2, results2);
@ -574,7 +693,7 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(); List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -624,7 +743,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -674,7 +793,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -724,7 +843,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -774,7 +893,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -824,7 +943,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -882,7 +1001,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -940,7 +1059,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -998,7 +1117,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1054,7 +1173,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1116,7 +1235,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1160,7 +1279,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1204,7 +1323,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1262,7 +1381,7 @@ public class TimeseriesQueryRunnerTest
); );
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList( Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, results); TestHelper.assertExpectedResults(expectedResults, results);
@ -1300,8 +1419,9 @@ public class TimeseriesQueryRunnerTest
) )
) )
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1340,8 +1460,9 @@ public class TimeseriesQueryRunnerTest
) )
) )
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1367,12 +1488,13 @@ public class TimeseriesQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
CONTEXT
), ),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1399,12 +1521,13 @@ public class TimeseriesQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
CONTEXT
), ),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1460,12 +1583,13 @@ public class TimeseriesQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
CONTEXT
), ),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1523,12 +1647,13 @@ public class TimeseriesQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
CONTEXT
), ),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults); TestHelper.assertExpectedResults(expectedResults, actualResults);

View File

@ -52,6 +52,7 @@ import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -166,8 +167,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -230,8 +231,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@ -295,8 +296,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@ -345,8 +346,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -394,8 +395,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -443,8 +444,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -485,8 +486,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -520,8 +521,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -569,8 +570,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -622,8 +623,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -664,8 +665,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -682,7 +683,7 @@ public class TopNQueryRunnerTest
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults( TestHelper.assertExpectedResults(
Lists.<Result<TopNResultValue>>newArrayList( Lists.<Result<TopNResultValue>>newArrayList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
@ -690,7 +691,7 @@ public class TopNQueryRunnerTest
new TopNResultValue(Lists.<Map<String, Object>>newArrayList()) new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
) )
), ),
runner.run(query) runner.run(query, context)
); );
} }
@ -721,7 +722,7 @@ public class TopNQueryRunnerTest
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults( TestHelper.assertExpectedResults(
Lists.<Result<TopNResultValue>>newArrayList( Lists.<Result<TopNResultValue>>newArrayList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
@ -729,7 +730,7 @@ public class TopNQueryRunnerTest
new TopNResultValue(Lists.<Map<String, Object>>newArrayList()) new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
) )
), ),
runner.run(query) runner.run(query, context)
); );
} }
@ -747,7 +748,7 @@ public class TopNQueryRunnerTest
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults( TestHelper.assertExpectedResults(
Sequences.toList( Sequences.toList(
runner.run( runner.run(
@ -761,9 +762,10 @@ public class TopNQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
context
), Lists.<Result<TopNResultValue>>newArrayList() ), Lists.<Result<TopNResultValue>>newArrayList()
), runner.run(query) ), runner.run(query, context)
); );
} }
@ -781,7 +783,7 @@ public class TopNQueryRunnerTest
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build(); .build();
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults( TestHelper.assertExpectedResults(
Sequences.toList( Sequences.toList(
runner.run( runner.run(
@ -795,10 +797,11 @@ public class TopNQueryRunnerTest
.intervals(QueryRunnerTestHelper.firstToThird) .intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators) .aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.build() .build(),
context
), Lists.<Result<TopNResultValue>>newArrayList() ), Lists.<Result<TopNResultValue>>newArrayList()
) )
, runner.run(query) , runner.run(query, context)
); );
} }
@ -840,8 +843,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -889,8 +892,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -945,8 +948,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -993,8 +996,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1034,8 +1037,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1075,8 +1078,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1116,8 +1119,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1157,8 +1160,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1209,8 +1212,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1261,8 +1264,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1313,8 +1316,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1358,8 +1361,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@ -1404,8 +1407,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1449,8 +1452,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1498,8 +1501,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1583,8 +1586,8 @@ public class TopNQueryRunnerTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -1666,6 +1669,7 @@ public class TopNQueryRunnerTest
) )
) )
); );
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
} }

View File

@ -42,6 +42,7 @@ import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -173,8 +174,8 @@ public class TopNUnionQueryTest
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }

View File

@ -177,7 +177,8 @@ public class AppendTest
.dataSource(dataSource) .dataSource(dataSource)
.build(); .build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -201,7 +202,8 @@ public class AppendTest
.dataSource(dataSource) .dataSource(dataSource)
.build(); .build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -225,7 +227,8 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery(); TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -249,7 +252,8 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery(); TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -273,7 +277,8 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery(); TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -297,7 +302,8 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery(); TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -343,7 +349,8 @@ public class AppendTest
TopNQuery query = makeTopNQuery(); TopNQuery query = makeTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -389,7 +396,8 @@ public class AppendTest
TopNQuery query = makeTopNQuery(); TopNQuery query = makeTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -416,7 +424,8 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery(); TopNQuery query = makeFilteredTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -433,7 +442,8 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery(); TopNQuery query = makeFilteredTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -455,7 +465,8 @@ public class AppendTest
SearchQuery query = makeSearchQuery(); SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -476,7 +487,8 @@ public class AppendTest
SearchQuery query = makeSearchQuery(); SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -496,7 +508,8 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery(); SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -517,7 +530,8 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery(); SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -558,7 +572,8 @@ public class AppendTest
.postAggregators(Arrays.<PostAggregator>asList(addRowsIndexConstant)) .postAggregators(Arrays.<PostAggregator>asList(addRowsIndexConstant))
.build(); .build();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
private TimeseriesQuery makeTimeseriesQuery() private TimeseriesQuery makeTimeseriesQuery()

View File

@ -1386,8 +1386,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " timeseries "; failMsg += " timeseries ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1419,8 +1420,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " filtered timeseries "; failMsg += " filtered timeseries ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TimeseriesResultValue>>newArrayList() Lists.<Result<TimeseriesResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1451,8 +1453,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " topN "; failMsg += " topN ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList( Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TopNResultValue>>newArrayList() Lists.<Result<TopNResultValue>>newArrayList()
); );
@ -1484,8 +1487,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " filtered topN "; failMsg += " filtered topN ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList( Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TopNResultValue>>newArrayList() Lists.<Result<TopNResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1501,8 +1505,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " search "; failMsg += " search ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList( Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SearchResultValue>>newArrayList() Lists.<Result<SearchResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1519,8 +1524,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " filtered search "; failMsg += " filtered search ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList( Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<SearchResultValue>>newArrayList() Lists.<Result<SearchResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1537,8 +1543,9 @@ public class SchemalessTestFull
.build(); .build();
failMsg += " timeBoundary "; failMsg += " timeBoundary ";
HashMap<String,Object> context = new HashMap<String, Object>();
Iterable<Result<TimeBoundaryResultValue>> actualResults = Sequences.toList( Iterable<Result<TimeBoundaryResultValue>> actualResults = Sequences.toList(
runner.run(query), runner.run(query, context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList() Lists.<Result<TimeBoundaryResultValue>>newArrayList()
); );
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);

View File

@ -60,6 +60,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
/** /**
@ -164,7 +165,8 @@ public class SchemalessTestSimple
) )
); );
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@ -237,7 +239,8 @@ public class SchemalessTestSimple
); );
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -265,7 +268,8 @@ public class SchemalessTestSimple
); );
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
@Test @Test
@ -290,6 +294,7 @@ public class SchemalessTestSimple
); );
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
} }

View File

@ -25,6 +25,8 @@ import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
@ -36,10 +38,14 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
/** @RunWith(Parameterized.class)
*/ public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest
public class CompressedFloatsIndexedSupplierTest
{ {
public CompressedFloatsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
super(compressionStrategy);
}
private IndexedFloats indexed; private IndexedFloats indexed;
private CompressedFloatsIndexedSupplier supplier; private CompressedFloatsIndexedSupplier supplier;
private float[] vals; private float[] vals;
@ -68,7 +74,8 @@ public class CompressedFloatsIndexedSupplierTest
supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer(
FloatBuffer.wrap(vals), FloatBuffer.wrap(vals),
5, 5,
ByteOrder.nativeOrder() ByteOrder.nativeOrder(),
compressionStrategy
); );
indexed = supplier.get(); indexed = supplier.get();
@ -82,7 +89,7 @@ public class CompressedFloatsIndexedSupplierTest
ByteArrayOutputStream baos = new ByteArrayOutputStream(); ByteArrayOutputStream baos = new ByteArrayOutputStream();
final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer(
FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder() FloatBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy
); );
theSupplier.writeToChannel(Channels.newChannel(baos)); theSupplier.writeToChannel(Channels.newChannel(baos));

View File

@ -23,6 +23,8 @@ import com.google.common.io.OutputSupplier;
import io.druid.collections.ResourceHolder; import io.druid.collections.ResourceHolder;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
@ -31,21 +33,31 @@ import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.nio.FloatBuffer; import java.nio.FloatBuffer;
/** @RunWith(Parameterized.class)
*/ public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyTest
public class CompressedFloatsSupplierSerializerTest
{ {
public CompressedFloatsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
super(compressionStrategy);
}
@Test @Test
public void testSanity() throws Exception public void testSanity() throws Exception
{ {
final ByteOrder order = ByteOrder.nativeOrder(); final ByteOrder order = ByteOrder.nativeOrder();
final int sizePer = 999;
CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer( CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer(
999, sizePer,
new GenericIndexedWriter<ResourceHolder<FloatBuffer>>( new GenericIndexedWriter<ResourceHolder<FloatBuffer>>(
new IOPeonForTesting(), new IOPeonForTesting(),
"test", "test",
CompressedFloatBufferObjectStrategy.getBufferForOrder(order) CompressedFloatBufferObjectStrategy.getBufferForOrder(
order,
compressionStrategy,
sizePer
) )
),
compressionStrategy
); );
serializer.open(); serializer.open();

View File

@ -25,6 +25,8 @@ import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
@ -36,10 +38,14 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
/** @RunWith(Parameterized.class)
*/ public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest
public class CompressedLongsIndexedSupplierTest
{ {
public CompressedLongsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
super(compressionStrategy);
}
private IndexedLongs indexed; private IndexedLongs indexed;
private CompressedLongsIndexedSupplier supplier; private CompressedLongsIndexedSupplier supplier;
private long[] vals; private long[] vals;
@ -66,7 +72,8 @@ public class CompressedLongsIndexedSupplierTest
supplier = CompressedLongsIndexedSupplier.fromLongBuffer( supplier = CompressedLongsIndexedSupplier.fromLongBuffer(
LongBuffer.wrap(vals), LongBuffer.wrap(vals),
5, 5,
ByteOrder.nativeOrder() ByteOrder.nativeOrder(),
compressionStrategy
); );
indexed = supplier.get(); indexed = supplier.get();
@ -78,7 +85,7 @@ public class CompressedLongsIndexedSupplierTest
ByteArrayOutputStream baos = new ByteArrayOutputStream(); ByteArrayOutputStream baos = new ByteArrayOutputStream();
final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer(
LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder() LongBuffer.wrap(vals), 5, ByteOrder.nativeOrder(), compressionStrategy
); );
theSupplier.writeToChannel(Channels.newChannel(baos)); theSupplier.writeToChannel(Channels.newChannel(baos));

View File

@ -23,6 +23,8 @@ import com.google.common.io.OutputSupplier;
import io.druid.collections.ResourceHolder; import io.druid.collections.ResourceHolder;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
@ -31,21 +33,27 @@ import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.nio.LongBuffer; import java.nio.LongBuffer;
/** @RunWith(Parameterized.class)
*/ public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTest
public class CompressedLongsSupplierSerializerTest
{ {
public CompressedLongsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
super(compressionStrategy);
}
@Test @Test
public void testSanity() throws Exception public void testSanity() throws Exception
{ {
final ByteOrder order = ByteOrder.nativeOrder(); final ByteOrder order = ByteOrder.nativeOrder();
final int sizePer = 999;
CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer( CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer(
999, sizePer,
new GenericIndexedWriter<ResourceHolder<LongBuffer>>( new GenericIndexedWriter<ResourceHolder<LongBuffer>>(
new IOPeonForTesting(), new IOPeonForTesting(),
"test", "test",
CompressedLongBufferObjectStrategy.getBufferForOrder(order) CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer)
) ),
compressionStrategy
); );
serializer.open(); serializer.open();

View File

@ -0,0 +1,52 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment.data;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import org.junit.runners.Parameterized;
import java.util.Arrays;
public class CompressionStrategyTest
{
@Parameterized.Parameters
public static Iterable<Object[]> compressionStrategies()
{
return Iterables.transform(
Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()),
new Function<CompressedObjectStrategy.CompressionStrategy, Object[]>()
{
@Override
public Object[] apply(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
return new Object[]{compressionStrategy};
}
}
);
}
protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy;
public CompressionStrategyTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy)
{
this.compressionStrategy = compressionStrategy;
}
}

View File

@ -62,6 +62,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
@ -449,8 +450,8 @@ public class SpatialFilterBonusTest
factory.createRunner(segment), factory.createRunner(segment),
factory.getToolchest() factory.getToolchest()
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -536,8 +537,8 @@ public class SpatialFilterBonusTest
factory.createRunner(segment), factory.createRunner(segment),
factory.getToolchest() factory.getToolchest()
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -62,6 +62,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
@ -479,8 +480,8 @@ public class SpatialFilterTest
factory.createRunner(segment), factory.createRunner(segment),
factory.getToolchest() factory.getToolchest()
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
@ -566,8 +567,8 @@ public class SpatialFilterTest
factory.createRunner(segment), factory.createRunner(segment),
factory.getToolchest() factory.getToolchest()
); );
HashMap<String,Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }
catch (Exception e) { catch (Exception e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -9,7 +9,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

View File

@ -28,7 +28,7 @@
<parent> <parent>
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid</artifactId> <artifactId>druid</artifactId>
<version>0.6.157-SNAPSHOT</version> <version>0.7.0-SNAPSHOT</version>
</parent> </parent>
<dependencies> <dependencies>

Some files were not shown because too many files have changed in this diff Show More