mirror of https://github.com/apache/druid.git
Merge branch 'master' into stop_task
Conflicts: merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java merger/src/main/java/com/metamx/druid/merger/coordinator/ExecutorServiceTaskRunner.java merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java
This commit is contained in:
commit
a0b2a34845
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.3.34-SNAPSHOT</version>
|
<version>0.3.35-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -31,7 +31,6 @@ import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||||
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -43,14 +42,16 @@ import java.util.Map;
|
||||||
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
|
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
||||||
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
||||||
@JsonSubTypes.Type(name = "groupBy", value= GroupByQuery.class),
|
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
|
||||||
@JsonSubTypes.Type(name = "segmentMetadata", value= SegmentMetadataQuery.class)
|
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class)
|
||||||
})
|
})
|
||||||
public interface Query<T>
|
public interface Query<T>
|
||||||
{
|
{
|
||||||
public static final String TIMESERIES = "timeseries";
|
public static final String TIMESERIES = "timeseries";
|
||||||
public static final String SEARCH = "search";
|
public static final String SEARCH = "search";
|
||||||
public static final String TIME_BOUNDARY = "timeBoundary";
|
public static final String TIME_BOUNDARY = "timeBoundary";
|
||||||
|
public static final String GROUP_BY = "groupBy";
|
||||||
|
public static final String SEGMENT_METADATA = "segmentMetadata";
|
||||||
|
|
||||||
public String getDataSource();
|
public String getDataSource();
|
||||||
|
|
||||||
|
|
|
@ -50,9 +50,7 @@ import com.metamx.metrics.MonitorSchedulerConfig;
|
||||||
import com.metamx.metrics.SysMonitor;
|
import com.metamx.metrics.SysMonitor;
|
||||||
import com.metamx.phonebook.PhoneBook;
|
import com.metamx.phonebook.PhoneBook;
|
||||||
import org.I0Itec.zkclient.ZkClient;
|
import org.I0Itec.zkclient.ZkClient;
|
||||||
|
import org.joda.time.Duration;
|
||||||
|
|
||||||
|
|
||||||
import org.mortbay.jetty.Server;
|
import org.mortbay.jetty.Server;
|
||||||
import org.skife.config.ConfigurationObjectFactory;
|
import org.skife.config.ConfigurationObjectFactory;
|
||||||
|
|
||||||
|
@ -332,9 +330,15 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
|
||||||
private void initializeEmitter()
|
private void initializeEmitter()
|
||||||
{
|
{
|
||||||
if (emitter == null) {
|
if (emitter == null) {
|
||||||
final HttpClient httpClient = HttpClientInit.createClient(
|
final HttpClientConfig.Builder configBuilder = HttpClientConfig.builder()
|
||||||
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
.withNumConnections(1);
|
||||||
);
|
|
||||||
|
final String emitterTimeoutDuration = props.getProperty("druid.emitter.timeOut");
|
||||||
|
if (emitterTimeoutDuration != null) {
|
||||||
|
configBuilder.withReadTimeout(new Duration(emitterTimeoutDuration));
|
||||||
|
}
|
||||||
|
|
||||||
|
final HttpClient httpClient = HttpClientInit.createClient(configBuilder.build(), lifecycle);
|
||||||
|
|
||||||
setEmitter(
|
setEmitter(
|
||||||
new ServiceEmitter(
|
new ServiceEmitter(
|
||||||
|
@ -358,7 +362,7 @@ public abstract class QueryableNode<T extends QueryableNode> extends Registering
|
||||||
@LifecycleStart
|
@LifecycleStart
|
||||||
public synchronized void start() throws Exception
|
public synchronized void start() throws Exception
|
||||||
{
|
{
|
||||||
if (! initialized) {
|
if (!initialized) {
|
||||||
init();
|
init();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.metamx.druid.index.v1.serde.Registererer;
|
import com.metamx.druid.index.v1.serde.Registererer;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -26,7 +26,6 @@ import com.metamx.druid.query.search.SearchHit;
|
||||||
import com.metamx.druid.query.search.SearchSortSpec;
|
import com.metamx.druid.query.search.SearchSortSpec;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
import com.metamx.druid.result.SearchResultValue;
|
import com.metamx.druid.result.SearchResultValue;
|
||||||
import org.joda.time.DateTime;
|
|
||||||
|
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
|
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.metamx.druid.query.QueryRunner;
|
||||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
|
@ -56,7 +56,6 @@ import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||||
import com.metamx.druid.result.BySegmentResultValueClass;
|
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -215,7 +214,6 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||||
descriptors.add(segment.rhs);
|
descriptors.add(segment.rhs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
return new LazySequence<T>(
|
return new LazySequence<T>(
|
||||||
new Supplier<Sequence<T>>()
|
new Supplier<Sequence<T>>()
|
||||||
{
|
{
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.google.common.base.Predicate;
|
||||||
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.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.metamx.common.Granularity;
|
import com.metamx.common.Granularity;
|
||||||
import com.metamx.druid.jackson.CommaListJoinDeserializer;
|
import com.metamx.druid.jackson.CommaListJoinDeserializer;
|
||||||
import com.metamx.druid.jackson.CommaListJoinSerializer;
|
import com.metamx.druid.jackson.CommaListJoinSerializer;
|
||||||
|
@ -36,9 +35,6 @@ import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||||
import com.metamx.druid.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
import com.metamx.druid.shard.ShardSpec;
|
import com.metamx.druid.shard.ShardSpec;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
|
|
@ -48,11 +48,6 @@ import com.metamx.http.client.io.AppendableByteArrayInputStream;
|
||||||
import com.metamx.http.client.response.ClientResponse;
|
import com.metamx.http.client.response.ClientResponse;
|
||||||
import com.metamx.http.client.response.InputStreamResponseHandler;
|
import com.metamx.http.client.response.InputStreamResponseHandler;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import org.jboss.netty.handler.codec.http.HttpChunk;
|
import org.jboss.netty.handler.codec.http.HttpChunk;
|
||||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid.client;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.metamx.phonebook.PhoneBook;
|
||||||
import com.metamx.phonebook.PhoneBookPeon;
|
import com.metamx.phonebook.PhoneBookPeon;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
|
|
@ -35,7 +35,6 @@ import org.I0Itec.zkclient.ZkClient;
|
||||||
import org.I0Itec.zkclient.exception.ZkNoNodeException;
|
import org.I0Itec.zkclient.exception.ZkNoNodeException;
|
||||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
||||||
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
|
||||||
import com.metamx.emitter.service.ServiceEventBuilder;
|
|
||||||
import net.spy.memcached.AddrUtil;
|
import net.spy.memcached.AddrUtil;
|
||||||
import net.spy.memcached.ConnectionFactoryBuilder;
|
import net.spy.memcached.ConnectionFactoryBuilder;
|
||||||
import net.spy.memcached.DefaultHashAlgorithm;
|
import net.spy.memcached.DefaultHashAlgorithm;
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -21,7 +21,6 @@ package com.metamx.druid.client.indexing;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
|
@ -31,7 +30,6 @@ import com.netflix.curator.x.discovery.ServiceInstance;
|
||||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -102,7 +100,6 @@ public class IndexingServiceClient
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private String baseUrl()
|
private String baseUrl()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -56,7 +56,6 @@ import com.metamx.metrics.Monitor;
|
||||||
import com.netflix.curator.framework.CuratorFramework;
|
import com.netflix.curator.framework.CuratorFramework;
|
||||||
import com.netflix.curator.x.discovery.ServiceDiscovery;
|
import com.netflix.curator.x.discovery.ServiceDiscovery;
|
||||||
|
|
||||||
|
|
||||||
import org.mortbay.jetty.servlet.Context;
|
import org.mortbay.jetty.servlet.Context;
|
||||||
import org.mortbay.jetty.servlet.ServletHolder;
|
import org.mortbay.jetty.servlet.ServletHolder;
|
||||||
import org.skife.config.ConfigurationObjectFactory;
|
import org.skife.config.ConfigurationObjectFactory;
|
||||||
|
@ -199,7 +198,6 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
||||||
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper());
|
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper());
|
||||||
lifecycle.addManagedInstance(baseClient);
|
lifecycle.addManagedInstance(baseClient);
|
||||||
|
|
||||||
|
|
||||||
final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, getEmitter(), baseClient);
|
final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, getEmitter(), baseClient);
|
||||||
|
|
||||||
List<Module> theModules = Lists.newArrayList();
|
List<Module> theModules = Lists.newArrayList();
|
||||||
|
|
|
@ -27,8 +27,6 @@ import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||||
import com.sun.jersey.guice.JerseyServletModule;
|
import com.sun.jersey.guice.JerseyServletModule;
|
||||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import javax.inject.Singleton;
|
import javax.inject.Singleton;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -48,7 +48,6 @@ public class FileRequestLogger implements RequestLogger
|
||||||
private volatile DateTime currentDay;
|
private volatile DateTime currentDay;
|
||||||
private volatile FileWriter fileWriter;
|
private volatile FileWriter fileWriter;
|
||||||
|
|
||||||
|
|
||||||
public FileRequestLogger(ObjectMapper objectMapper, ScheduledExecutorService exec, File baseDir)
|
public FileRequestLogger(ObjectMapper objectMapper, ScheduledExecutorService exec, File baseDir)
|
||||||
{
|
{
|
||||||
this.exec = exec;
|
this.exec = exec;
|
||||||
|
|
|
@ -34,7 +34,6 @@ import com.metamx.emitter.service.AlertEvent;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.mortbay.jetty.Request;
|
import org.mortbay.jetty.Request;
|
||||||
|
|
||||||
|
|
|
@ -117,7 +117,6 @@ public class Initialization
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Load properties.
|
* Load properties.
|
||||||
* Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper.
|
* Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper.
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.google.common.base.Function;
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
|
public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package com.metamx.druid.query;
|
package com.metamx.druid.query;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Joiner;
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class)
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class)
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid.query.dimension;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.extraction;
|
package com.metamx.druid.query.extraction;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.extraction;
|
package com.metamx.druid.query.extraction;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.extraction;
|
package com.metamx.druid.query.extraction;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
@ -75,4 +72,4 @@ public class RegexDimExtractionFn implements DimExtractionFn
|
||||||
{
|
{
|
||||||
return String.format("regex(%s)", expr);
|
return String.format("regex(%s)", expr);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.ibm.icu.text.SimpleDateFormat;
|
import com.ibm.icu.text.SimpleDateFormat;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
|
|
@ -22,9 +22,9 @@ package com.metamx.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -40,6 +40,8 @@ public class AndDimFilter implements DimFilter
|
||||||
@JsonProperty("fields") List<DimFilter> fields
|
@JsonProperty("fields") List<DimFilter> fields
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
fields.removeAll(Collections.singletonList(null));
|
||||||
|
Preconditions.checkArgument(fields.size() > 0, "AND operator requires at least one field");
|
||||||
this.fields = fields;
|
this.fields = fields;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.filter;
|
package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
|
||||||
|
|
|
@ -20,10 +20,8 @@
|
||||||
package com.metamx.druid.query.filter;
|
package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -21,10 +21,9 @@ package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -42,6 +41,10 @@ public class ExtractionDimFilter implements DimFilter
|
||||||
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
Preconditions.checkArgument(value != null, "value must not be null");
|
||||||
|
Preconditions.checkArgument(dimExtractionFn != null, "extraction function must not be null");
|
||||||
|
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
this.dimExtractionFn = dimExtractionFn;
|
this.dimExtractionFn = dimExtractionFn;
|
||||||
|
|
|
@ -3,6 +3,7 @@ package com.metamx.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Charsets;
|
import com.google.common.base.Charsets;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
@ -17,6 +18,8 @@ public class JavaScriptDimFilter implements DimFilter
|
||||||
@JsonProperty("function") String function
|
@JsonProperty("function") String function
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
Preconditions.checkArgument(function != null, "function must not be null");
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.function = function;
|
this.function = function;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,11 +19,9 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.filter;
|
package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
@ -38,6 +36,7 @@ public class NotDimFilter implements DimFilter
|
||||||
@JsonProperty("field") DimFilter field
|
@JsonProperty("field") DimFilter field
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(field != null, "NOT operator requires at least one field");
|
||||||
this.field = field;
|
this.field = field;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,9 +22,9 @@ package com.metamx.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -40,6 +40,8 @@ public class OrDimFilter implements DimFilter
|
||||||
@JsonProperty("fields") List<DimFilter> fields
|
@JsonProperty("fields") List<DimFilter> fields
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
fields.removeAll(Collections.singletonList(null));
|
||||||
|
Preconditions.checkArgument(fields.size() > 0, "OR operator requires at least one field");
|
||||||
this.fields = fields;
|
this.fields = fields;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,8 +22,7 @@ package com.metamx.druid.query.filter;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Charsets;
|
import com.google.common.base.Charsets;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
@ -40,6 +39,8 @@ public class RegexDimFilter implements DimFilter
|
||||||
@JsonProperty("pattern") String pattern
|
@JsonProperty("pattern") String pattern
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
Preconditions.checkArgument(pattern != null, "pattern must not be null");
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.pattern = pattern;
|
this.pattern = pattern;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,9 +21,9 @@ package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Charsets;
|
import com.google.common.base.Charsets;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -38,6 +38,9 @@ public class SearchQueryDimFilter implements DimFilter
|
||||||
@JsonProperty("query") SearchQuerySpec query
|
@JsonProperty("query") SearchQuerySpec query
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
Preconditions.checkArgument(query != null, "query must not be null");
|
||||||
|
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.query = query;
|
this.query = query;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,11 +19,9 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.filter;
|
package com.metamx.druid.query.filter;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
@ -40,6 +38,8 @@ public class SelectorDimFilter implements DimFilter
|
||||||
@JsonProperty("value") String value
|
@JsonProperty("value") String value
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
Preconditions.checkArgument(dimension != null, "dimension must not be null");
|
||||||
|
Preconditions.checkArgument(value != null, "value must not be null");
|
||||||
this.dimension = dimension;
|
this.dimension = dimension;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,14 +30,13 @@ import com.metamx.druid.QueryGranularity;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.aggregation.post.PostAggregator;
|
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||||
import com.metamx.druid.input.Row;
|
import com.metamx.druid.input.Row;
|
||||||
|
import com.metamx.druid.query.Queries;
|
||||||
import com.metamx.druid.query.dimension.DefaultDimensionSpec;
|
import com.metamx.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import com.metamx.druid.query.dimension.DimensionSpec;
|
import com.metamx.druid.query.dimension.DimensionSpec;
|
||||||
import com.metamx.druid.query.filter.DimFilter;
|
import com.metamx.druid.query.filter.DimFilter;
|
||||||
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
||||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -77,6 +76,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
||||||
|
|
||||||
Preconditions.checkNotNull(this.granularity, "Must specify a granularity");
|
Preconditions.checkNotNull(this.granularity, "Must specify a granularity");
|
||||||
Preconditions.checkNotNull(this.aggregatorSpecs, "Must specify at least one aggregator");
|
Preconditions.checkNotNull(this.aggregatorSpecs, "Must specify at least one aggregator");
|
||||||
|
Queries.verifyAggregations(this.aggregatorSpecs, this.postAggregatorSpecs);
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("filter")
|
@JsonProperty("filter")
|
||||||
|
@ -118,11 +118,11 @@ public class GroupByQuery extends BaseQuery<Row>
|
||||||
@Override
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return "groupBy";
|
return Query.GROUP_BY;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Query withOverriddenContext(Map<String, String> contextOverride)
|
public GroupByQuery withOverriddenContext(Map<String, String> contextOverride)
|
||||||
{
|
{
|
||||||
return new GroupByQuery(
|
return new GroupByQuery(
|
||||||
getDataSource(),
|
getDataSource(),
|
||||||
|
@ -137,7 +137,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Query withQuerySegmentSpec(QuerySegmentSpec spec)
|
public GroupByQuery withQuerySegmentSpec(QuerySegmentSpec spec)
|
||||||
{
|
{
|
||||||
return new GroupByQuery(
|
return new GroupByQuery(
|
||||||
getDataSource(),
|
getDataSource(),
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Functions;
|
import com.google.common.base.Functions;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
|
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.ISE;
|
||||||
import com.metamx.common.guava.Accumulator;
|
import com.metamx.common.guava.Accumulator;
|
||||||
|
@ -49,6 +50,7 @@ import org.joda.time.Minutes;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,6 +59,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
||||||
{
|
{
|
||||||
|
|
||||||
private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>(){};
|
private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>(){};
|
||||||
|
private static final String GROUP_BY_MERGE_KEY = "groupByMerge";
|
||||||
|
private static final Map<String, String> NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false");
|
||||||
|
|
||||||
private static final int maxRows;
|
private static final int maxRows;
|
||||||
|
|
||||||
|
@ -75,77 +79,85 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> run(Query<Row> input)
|
public Sequence<Row> run(Query<Row> input)
|
||||||
{
|
{
|
||||||
final GroupByQuery query = (GroupByQuery) input;
|
if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
||||||
|
return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner);
|
||||||
final QueryGranularity gran = query.getGranularity();
|
}
|
||||||
final long timeStart = query.getIntervals().get(0).getStartMillis();
|
else {
|
||||||
|
return runner.run(input);
|
||||||
// use gran.iterable instead of gran.truncate so that
|
}
|
||||||
// AllGranularity returns timeStart instead of Long.MIN_VALUE
|
|
||||||
final long granTimeStart = gran.iterable(timeStart, timeStart+1).iterator().next();
|
|
||||||
|
|
||||||
final List<AggregatorFactory> aggs = Lists.transform(
|
|
||||||
query.getAggregatorSpecs(),
|
|
||||||
new Function<AggregatorFactory, AggregatorFactory>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public AggregatorFactory apply(@Nullable AggregatorFactory input)
|
|
||||||
{
|
|
||||||
return input.getCombiningFactory();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
final List<String> dimensions = Lists.transform(
|
|
||||||
query.getDimensions(),
|
|
||||||
new Function<DimensionSpec, String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String apply(@Nullable DimensionSpec input)
|
|
||||||
{
|
|
||||||
return input.getOutputName();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
final IncrementalIndex index = runner.run(query).accumulate(
|
|
||||||
new IncrementalIndex(
|
|
||||||
// use granularity truncated min timestamp
|
|
||||||
// since incoming truncated timestamps may precede timeStart
|
|
||||||
granTimeStart,
|
|
||||||
gran,
|
|
||||||
aggs.toArray(new AggregatorFactory[aggs.size()])
|
|
||||||
),
|
|
||||||
new Accumulator<IncrementalIndex, Row>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in)
|
|
||||||
{
|
|
||||||
if (accumulated.add(Rows.toInputRow(in, dimensions)) > maxRows) {
|
|
||||||
throw new ISE("Computation exceeds maxRows limit[%s]", maxRows);
|
|
||||||
}
|
|
||||||
|
|
||||||
return accumulated;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
// convert millis back to timestamp according to granularity to preserve time zone information
|
|
||||||
return Sequences.map(
|
|
||||||
Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())),
|
|
||||||
new Function<Row, Row>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Row apply(Row input)
|
|
||||||
{
|
|
||||||
final MapBasedRow row = (MapBasedRow) input;
|
|
||||||
return new MapBasedRow(gran.toDateTime(row.getTimestampFromEpoch()), row.getEvent());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Sequence<Row> mergeGroupByResults(GroupByQuery query, QueryRunner<Row> runner)
|
||||||
|
{
|
||||||
|
final QueryGranularity gran = query.getGranularity();
|
||||||
|
final long timeStart = query.getIntervals().get(0).getStartMillis();
|
||||||
|
|
||||||
|
// use gran.iterable instead of gran.truncate so that
|
||||||
|
// AllGranularity returns timeStart instead of Long.MIN_VALUE
|
||||||
|
final long granTimeStart = gran.iterable(timeStart, timeStart+1).iterator().next();
|
||||||
|
|
||||||
|
final List<AggregatorFactory> aggs = Lists.transform(
|
||||||
|
query.getAggregatorSpecs(),
|
||||||
|
new Function<AggregatorFactory, AggregatorFactory>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory apply(@Nullable AggregatorFactory input)
|
||||||
|
{
|
||||||
|
return input.getCombiningFactory();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
final List<String> dimensions = Lists.transform(
|
||||||
|
query.getDimensions(),
|
||||||
|
new Function<DimensionSpec, String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public String apply(@Nullable DimensionSpec input)
|
||||||
|
{
|
||||||
|
return input.getOutputName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
final IncrementalIndex index = runner.run(query).accumulate(
|
||||||
|
new IncrementalIndex(
|
||||||
|
// use granularity truncated min timestamp
|
||||||
|
// since incoming truncated timestamps may precede timeStart
|
||||||
|
granTimeStart,
|
||||||
|
gran,
|
||||||
|
aggs.toArray(new AggregatorFactory[aggs.size()])
|
||||||
|
),
|
||||||
|
new Accumulator<IncrementalIndex, Row>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in)
|
||||||
|
{
|
||||||
|
if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions)) > maxRows) {
|
||||||
|
throw new ISE("Computation exceeds maxRows limit[%s]", maxRows);
|
||||||
|
}
|
||||||
|
|
||||||
|
return accumulated;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
// convert millis back to timestamp according to granularity to preserve time zone information
|
||||||
|
return Sequences.map(
|
||||||
|
Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())),
|
||||||
|
new Function<Row, Row>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Row apply(Row input)
|
||||||
|
{
|
||||||
|
final MapBasedRow row = (MapBasedRow) input;
|
||||||
|
return new MapBasedRow(gran.toDateTime(row.getTimestampFromEpoch()), row.getEvent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> mergeSequences(Sequence<Sequence<Row>> seqOfSequences)
|
public Sequence<Row> mergeSequences(Sequence<Sequence<Row>> seqOfSequences)
|
||||||
{
|
{
|
||||||
|
|
|
@ -21,11 +21,8 @@ package com.metamx.druid.query.metadata;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import com.metamx.druid.index.column.ValueType;
|
import com.metamx.druid.index.column.ValueType;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class ColumnAnalysis
|
public class ColumnAnalysis
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.metadata;
|
package com.metamx.druid.query.metadata;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
|
||||||
|
|
|
@ -22,14 +22,9 @@ package com.metamx.druid.query.metadata;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Charsets;
|
import com.google.common.base.Charsets;
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.io.ByteArrayOutputStream;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
|
@ -19,8 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.metadata;
|
package com.metamx.druid.query.metadata;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
|
@ -19,12 +19,12 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.metadata;
|
package com.metamx.druid.query.metadata;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.metamx.druid.BaseQuery;
|
import com.metamx.druid.BaseQuery;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||||
|
@ -33,6 +33,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||||
private final ColumnIncluderator toInclude;
|
private final ColumnIncluderator toInclude;
|
||||||
private final boolean merge;
|
private final boolean merge;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
public SegmentMetadataQuery(
|
public SegmentMetadataQuery(
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||||
|
@ -68,7 +69,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||||
@Override
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return "segmentMetadata";
|
return Query.SEGMENT_METADATA;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -50,7 +50,6 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
|
||||||
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
||||||
{
|
{
|
||||||
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>(){};
|
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>(){};
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.search;
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
|
|
@ -19,12 +19,9 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.search;
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.search;
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
|
|
@ -32,8 +32,6 @@ import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
import com.metamx.druid.result.SearchResultValue;
|
import com.metamx.druid.result.SearchResultValue;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
|
@ -21,11 +21,6 @@ package com.metamx.druid.query.search;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.google.common.base.Predicate;
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.search;
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package com.metamx.druid.query.search;
|
package com.metamx.druid.query.search;
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -50,4 +49,4 @@ public class StrlenSearchSortSpec implements SearchSortSpec
|
||||||
{
|
{
|
||||||
return "stringLengthSort";
|
return "stringLengthSort";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.metamx.druid.Query;
|
||||||
import com.metamx.druid.query.QueryRunner;
|
import com.metamx.druid.query.QueryRunner;
|
||||||
import com.metamx.druid.utils.JodaUtils;
|
import com.metamx.druid.utils.JodaUtils;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
|
|
@ -27,7 +27,6 @@ import com.metamx.druid.Query;
|
||||||
import com.metamx.druid.query.QueryRunner;
|
import com.metamx.druid.query.QueryRunner;
|
||||||
import com.metamx.druid.utils.JodaUtils;
|
import com.metamx.druid.utils.JodaUtils;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
import com.metamx.druid.query.QueryRunner;
|
import com.metamx.druid.query.QueryRunner;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
|
@ -19,8 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.query.segment;
|
package com.metamx.druid.query.segment;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
|
@ -31,7 +31,6 @@ import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
import com.metamx.druid.result.TimeBoundaryResultValue;
|
import com.metamx.druid.result.TimeBoundaryResultValue;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid.query.timeseries;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.metamx.druid.BaseQuery;
|
import com.metamx.druid.BaseQuery;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
|
@ -35,10 +34,6 @@ import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||||
import com.metamx.druid.result.Result;
|
import com.metamx.druid.result.Result;
|
||||||
import com.metamx.druid.result.TimeseriesResultValue;
|
import com.metamx.druid.result.TimeseriesResultValue;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -47,8 +42,6 @@ import java.util.Map;
|
||||||
@JsonTypeName("timeseries")
|
@JsonTypeName("timeseries")
|
||||||
public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
||||||
{
|
{
|
||||||
public static final String TIMESERIES = "timeseries";
|
|
||||||
|
|
||||||
private final DimFilter dimFilter;
|
private final DimFilter dimFilter;
|
||||||
private final QueryGranularity granularity;
|
private final QueryGranularity granularity;
|
||||||
private final List<AggregatorFactory> aggregatorSpecs;
|
private final List<AggregatorFactory> aggregatorSpecs;
|
||||||
|
@ -83,7 +76,7 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
||||||
@Override
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return TIMESERIES;
|
return Query.TIMESERIES;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty("filter")
|
@JsonProperty("filter")
|
||||||
|
@ -135,7 +128,7 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
||||||
computeOverridenContext(contextOverrides)
|
computeOverridenContext(contextOverrides)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
|
|
|
@ -51,7 +51,6 @@ import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.Minutes;
|
import org.joda.time.Minutes;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
import org.joda.time.format.ISODateTimeFormat;
|
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.result;
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonValue;
|
import com.fasterxml.jackson.annotation.JsonValue;
|
||||||
import com.metamx.druid.query.search.SearchHit;
|
import com.metamx.druid.query.search.SearchHit;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.result;
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonValue;
|
import com.fasterxml.jackson.annotation.JsonValue;
|
||||||
|
|
||||||
|
|
|
@ -19,9 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.result;
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -99,4 +96,4 @@ public class Result<T> implements Comparable<Result<T>>
|
||||||
", value=" + value +
|
", value=" + value +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonValue;
|
import com.fasterxml.jackson.annotation.JsonValue;
|
||||||
import com.metamx.druid.query.search.SearchHit;
|
import com.metamx.druid.query.search.SearchHit;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonValue;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -67,7 +66,6 @@ public class TimeBoundaryResultValue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(Object o)
|
||||||
{
|
{
|
||||||
|
@ -111,4 +109,4 @@ public class TimeBoundaryResultValue
|
||||||
throw new IAE("Cannot get time from type[%s]", val.getClass());
|
throw new IAE("Cannot get time from type[%s]", val.getClass());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,8 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.result;
|
package com.metamx.druid.result;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
|
@ -5,7 +5,6 @@ import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.partition.LinearPartitionChunk;
|
import com.metamx.druid.partition.LinearPartitionChunk;
|
||||||
import com.metamx.druid.partition.PartitionChunk;
|
import com.metamx.druid.partition.PartitionChunk;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class LinearShardSpec implements ShardSpec {
|
public class LinearShardSpec implements ShardSpec {
|
||||||
|
@ -25,7 +24,6 @@ public class LinearShardSpec implements ShardSpec {
|
||||||
return partitionNum;
|
return partitionNum;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <T> PartitionChunk<T> createChunk(T obj) {
|
public <T> PartitionChunk<T> createChunk(T obj) {
|
||||||
return new LinearPartitionChunk<T>(partitionNum, obj);
|
return new LinearPartitionChunk<T>(partitionNum, obj);
|
||||||
|
@ -40,4 +38,4 @@ public class LinearShardSpec implements ShardSpec {
|
||||||
public boolean isInChunk(InputRow inputRow) {
|
public boolean isInChunk(InputRow inputRow) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.partition.PartitionChunk;
|
import com.metamx.druid.partition.PartitionChunk;
|
||||||
import com.metamx.druid.partition.SingleElementPartitionChunk;
|
import com.metamx.druid.partition.SingleElementPartitionChunk;
|
||||||
|
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.druid.input.InputRow;
|
import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.partition.PartitionChunk;
|
import com.metamx.druid.partition.PartitionChunk;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.partition.PartitionChunk;
|
import com.metamx.druid.partition.PartitionChunk;
|
||||||
import com.metamx.druid.partition.StringPartitionChunk;
|
import com.metamx.druid.partition.StringPartitionChunk;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,6 @@ import org.I0Itec.zkclient.exception.ZkMarshallingError;
|
||||||
import org.I0Itec.zkclient.serialize.ZkSerializer;
|
import org.I0Itec.zkclient.serialize.ZkSerializer;
|
||||||
|
|
||||||
import java.io.*;
|
import java.io.*;
|
||||||
import java.nio.charset.Charset;
|
|
||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||||
import com.metamx.druid.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
|
@ -61,7 +61,6 @@ public class MemcachedCacheBenchmark extends SimpleBenchmark
|
||||||
3600 // 1 hour
|
3600 // 1 hour
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
randBytes = new byte[objectSize * 1024];
|
randBytes = new byte[objectSize * 1024];
|
||||||
new Random(0).nextBytes(randBytes);
|
new Random(0).nextBytes(randBytes);
|
||||||
}
|
}
|
||||||
|
|
|
@ -96,7 +96,6 @@ public class SingleDimensionShardSpecTest
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
|
||||||
for (Map.Entry<SingleDimensionShardSpec, List<Pair<Boolean, Map<String, String>>>> entry : tests.entrySet()) {
|
for (Map.Entry<SingleDimensionShardSpec, List<Pair<Boolean, Map<String, String>>>> entry : tests.entrySet()) {
|
||||||
SingleDimensionShardSpec spec = entry.getKey();
|
SingleDimensionShardSpec spec = entry.getKey();
|
||||||
for (Pair<Boolean, Map<String, String>> pair : entry.getValue()) {
|
for (Pair<Boolean, Map<String, String>> pair : entry.getValue()) {
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.3.34-SNAPSHOT</version>
|
<version>0.3.35-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
|
@ -19,8 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid;
|
package com.metamx.druid;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.joda.time.Chronology;
|
import org.joda.time.Chronology;
|
||||||
|
@ -267,7 +265,6 @@ public class PeriodGranularity extends BaseQueryGranularity
|
||||||
return t;
|
return t;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private static boolean isCompoundPeriod(Period period)
|
private static boolean isCompoundPeriod(Period period)
|
||||||
{
|
{
|
||||||
int[] values = period.getValues();
|
int[] values = period.getValues();
|
||||||
|
@ -322,7 +319,6 @@ public class PeriodGranularity extends BaseQueryGranularity
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public byte[] cacheKey()
|
public byte[] cacheKey()
|
||||||
{
|
{
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.ReadableDuration;
|
import org.joda.time.ReadableDuration;
|
||||||
|
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.primitives.Floats;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
|
|
@ -44,7 +44,6 @@ public class HistogramAggregator implements Aggregator
|
||||||
|
|
||||||
private Histogram histogram;
|
private Histogram histogram;
|
||||||
|
|
||||||
|
|
||||||
public HistogramAggregator(String name, FloatMetricSelector selector, float[] breaks) {
|
public HistogramAggregator(String name, FloatMetricSelector selector, float[] breaks) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.selector = selector;
|
this.selector = selector;
|
||||||
|
|
|
@ -27,8 +27,6 @@ import com.google.common.primitives.Longs;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
import org.apache.commons.codec.binary.Base64;
|
import org.apache.commons.codec.binary.Base64;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package com.metamx.druid.aggregation;
|
package com.metamx.druid.aggregation;
|
||||||
|
|
||||||
|
|
||||||
import com.google.common.primitives.Floats;
|
import com.google.common.primitives.Floats;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import com.metamx.druid.processing.FloatMetricSelector;
|
import com.metamx.druid.processing.FloatMetricSelector;
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
||||||
public class HistogramVisual
|
public class HistogramVisual
|
||||||
|
|
|
@ -21,7 +21,6 @@ package com.metamx.druid.aggregation;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.druid.processing.FloatMetricSelector;
|
import com.metamx.druid.processing.FloatMetricSelector;
|
||||||
import org.mozilla.javascript.Context;
|
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -28,7 +28,6 @@ import com.metamx.common.IAE;
|
||||||
import com.metamx.druid.processing.FloatMetricSelector;
|
import com.metamx.druid.processing.FloatMetricSelector;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
import org.mozilla.javascript.Context;
|
import org.mozilla.javascript.Context;
|
||||||
import org.mozilla.javascript.ContextAction;
|
import org.mozilla.javascript.ContextAction;
|
||||||
import org.mozilla.javascript.ContextFactory;
|
import org.mozilla.javascript.ContextFactory;
|
||||||
|
@ -228,7 +227,6 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
|
||||||
final Function fnCombine = getScriptFunction("combine", scope);
|
final Function fnCombine = getScriptFunction("combine", scope);
|
||||||
Context.exit();
|
Context.exit();
|
||||||
|
|
||||||
|
|
||||||
return new JavaScriptAggregator.ScriptAggregator()
|
return new JavaScriptAggregator.ScriptAggregator()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.primitives.Longs;
|
import com.google.common.primitives.Longs;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.IAE;
|
import com.metamx.common.IAE;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue