mirror of https://github.com/apache/druid.git
Merge branch 'master' of github.com:metamx/druid
Conflicts: client/src/main/java/com/metamx/druid/QueryableNode.java server/src/main/java/com/metamx/druid/http/ComputeNode.java
This commit is contained in:
commit
120b822c18
|
@ -31,7 +31,6 @@ import com.metamx.druid.query.segment.QuerySegmentWalker;
|
|||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
||||
|
||||
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
|
|
@ -50,8 +50,6 @@ import com.metamx.metrics.MonitorSchedulerConfig;
|
|||
import com.metamx.metrics.SysMonitor;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import org.I0Itec.zkclient.ZkClient;
|
||||
|
||||
|
||||
import org.joda.time.Duration;
|
||||
import org.mortbay.jetty.Server;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.metamx.druid.index.v1.serde.Registererer;
|
||||
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -30,7 +30,6 @@ import com.metamx.druid.query.QueryRunner;
|
|||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Iterator;
|
||||
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.Result;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -215,7 +214,6 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
descriptors.add(segment.rhs);
|
||||
}
|
||||
|
||||
|
||||
return new LazySequence<T>(
|
||||
new Supplier<Sequence<T>>()
|
||||
{
|
||||
|
|
|
@ -36,9 +36,6 @@ import com.metamx.druid.query.segment.SegmentDescriptor;
|
|||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.druid.shard.ShardSpec;
|
||||
|
||||
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
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.InputStreamResponseHandler;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import org.jboss.netty.handler.codec.http.HttpChunk;
|
||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||
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.google.common.collect.Maps;
|
||||
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.I0Itec.zkclient.ZkClient;
|
|||
import org.I0Itec.zkclient.exception.ZkNoNodeException;
|
||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
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.ConnectionFactoryBuilder;
|
||||
import net.spy.memcached.DefaultHashAlgorithm;
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,7 +31,6 @@ import com.netflix.curator.x.discovery.ServiceInstance;
|
|||
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.util.Iterator;
|
||||
|
@ -102,7 +101,6 @@ public class IndexingServiceClient
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private String baseUrl()
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -56,7 +56,6 @@ import com.metamx.metrics.Monitor;
|
|||
import com.netflix.curator.framework.CuratorFramework;
|
||||
import com.netflix.curator.x.discovery.ServiceDiscovery;
|
||||
|
||||
|
||||
import org.mortbay.jetty.servlet.Context;
|
||||
import org.mortbay.jetty.servlet.ServletHolder;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
@ -199,7 +198,6 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper());
|
||||
lifecycle.addManagedInstance(baseClient);
|
||||
|
||||
|
||||
final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, getEmitter(), baseClient);
|
||||
|
||||
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.spi.container.servlet.GuiceContainer;
|
||||
|
||||
|
||||
|
||||
import javax.inject.Singleton;
|
||||
|
||||
/**
|
||||
|
|
|
@ -48,7 +48,6 @@ public class FileRequestLogger implements RequestLogger
|
|||
private volatile DateTime currentDay;
|
||||
private volatile FileWriter fileWriter;
|
||||
|
||||
|
||||
public FileRequestLogger(ObjectMapper objectMapper, ScheduledExecutorService exec, File baseDir)
|
||||
{
|
||||
this.exec = exec;
|
||||
|
|
|
@ -34,7 +34,6 @@ import com.metamx.emitter.service.AlertEvent;
|
|||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.mortbay.jetty.Request;
|
||||
|
||||
|
|
|
@ -117,7 +117,6 @@ public class Initialization
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Load properties.
|
||||
* 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.druid.Query;
|
||||
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
|
||||
|
||||
|
||||
/**
|
||||
*/
|
||||
@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.metamx.druid.query.extraction.DimExtractionFn;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid.query.dimension;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.metamx.common.IAE;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.ibm.icu.text.SimpleDateFormat;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.text.ParseException;
|
||||
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.JsonProperty;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -40,6 +40,8 @@ public class AndDimFilter implements DimFilter
|
|||
@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;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -21,10 +21,9 @@ package com.metamx.druid.query.filter;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
|
@ -42,6 +41,10 @@ public class ExtractionDimFilter implements DimFilter
|
|||
@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.value = value;
|
||||
this.dimExtractionFn = dimExtractionFn;
|
||||
|
|
|
@ -3,6 +3,7 @@ package com.metamx.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -17,6 +18,8 @@ public class JavaScriptDimFilter implements DimFilter
|
|||
@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.function = function;
|
||||
}
|
||||
|
|
|
@ -19,11 +19,9 @@
|
|||
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -38,6 +36,7 @@ public class NotDimFilter implements DimFilter
|
|||
@JsonProperty("field") DimFilter field
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(field != null, "NOT operator requires at least one 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.JsonProperty;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -40,6 +40,8 @@ public class OrDimFilter implements DimFilter
|
|||
@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;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,8 +22,7 @@ package com.metamx.druid.query.filter;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -40,6 +39,8 @@ public class RegexDimFilter implements DimFilter
|
|||
@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.pattern = pattern;
|
||||
}
|
||||
|
|
|
@ -21,9 +21,9 @@ package com.metamx.druid.query.filter;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
|
@ -38,6 +38,9 @@ public class SearchQueryDimFilter implements DimFilter
|
|||
@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.query = query;
|
||||
}
|
||||
|
|
|
@ -19,11 +19,9 @@
|
|||
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -40,6 +38,8 @@ public class SelectorDimFilter implements DimFilter
|
|||
@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.value = value;
|
||||
}
|
||||
|
|
|
@ -36,8 +36,6 @@ import com.metamx.druid.query.filter.DimFilter;
|
|||
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
|
||||
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnAnalysis
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.metadata;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -26,8 +26,6 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.metadata;
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.metamx.druid.BaseQuery;
|
|||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||
|
|
|
@ -50,7 +50,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
||||
{
|
||||
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.collect.Lists;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
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.SearchResultValue;
|
||||
|
||||
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
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;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.metamx.druid.Query;
|
|||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.utils.JodaUtils;
|
||||
|
||||
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Collections;
|
||||
|
|
|
@ -27,7 +27,6 @@ import com.metamx.druid.Query;
|
|||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.utils.JodaUtils;
|
||||
|
||||
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
|
||||
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.query.segment;
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
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.TimeBoundaryResultValue;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
|
|
@ -35,10 +35,6 @@ import com.metamx.druid.query.segment.QuerySegmentSpec;
|
|||
import com.metamx.druid.result.Result;
|
||||
import com.metamx.druid.result.TimeseriesResultValue;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.result;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import com.metamx.druid.query.search.SearchHit;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.result;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.result;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.DateTime;
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import com.metamx.druid.query.search.SearchHit;
|
||||
|
||||
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonValue;
|
|||
import com.metamx.common.IAE;
|
||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Map;
|
||||
|
@ -67,7 +66,6 @@ public class TimeBoundaryResultValue
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.result;
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
|
||||
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.PartitionChunk;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class LinearShardSpec implements ShardSpec {
|
||||
|
@ -25,7 +24,6 @@ public class LinearShardSpec implements ShardSpec {
|
|||
return partitionNum;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public <T> PartitionChunk<T> createChunk(T obj) {
|
||||
return new LinearPartitionChunk<T>(partitionNum, obj);
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.metamx.druid.input.InputRow;
|
|||
import com.metamx.druid.partition.PartitionChunk;
|
||||
import com.metamx.druid.partition.SingleElementPartitionChunk;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
|||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.partition.PartitionChunk;
|
||||
|
||||
|
||||
|
||||
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.StringPartitionChunk;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
|
|
@ -30,7 +30,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
|||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
|
|
|
@ -61,7 +61,6 @@ public class MemcachedCacheBenchmark extends SimpleBenchmark
|
|||
3600 // 1 hour
|
||||
);
|
||||
|
||||
|
||||
randBytes = new byte[objectSize * 1024];
|
||||
new Random(0).nextBytes(randBytes);
|
||||
}
|
||||
|
|
|
@ -96,7 +96,6 @@ public class SingleDimensionShardSpecTest
|
|||
)
|
||||
.build();
|
||||
|
||||
|
||||
for (Map.Entry<SingleDimensionShardSpec, List<Pair<Boolean, Map<String, String>>>> entry : tests.entrySet()) {
|
||||
SingleDimensionShardSpec spec = entry.getKey();
|
||||
for (Pair<Boolean, Map<String, String>> pair : entry.getValue()) {
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.primitives.Longs;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid;
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Chronology;
|
||||
|
@ -267,7 +265,6 @@ public class PeriodGranularity extends BaseQueryGranularity
|
|||
return t;
|
||||
}
|
||||
|
||||
|
||||
private static boolean isCompoundPeriod(Period period)
|
||||
{
|
||||
int[] values = period.getValues();
|
||||
|
@ -322,7 +319,6 @@ public class PeriodGranularity extends BaseQueryGranularity
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public byte[] cacheKey()
|
||||
{
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.metamx.common.IAE;
|
||||
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.ReadableDuration;
|
||||
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.primitives.Doubles;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
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.Longs;
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
|
||||
|
|
|
@ -44,7 +44,6 @@ public class HistogramAggregator implements Aggregator
|
|||
|
||||
private Histogram histogram;
|
||||
|
||||
|
||||
public HistogramAggregator(String name, FloatMetricSelector selector, float[] breaks) {
|
||||
this.name = name;
|
||||
this.selector = selector;
|
||||
|
|
|
@ -27,8 +27,6 @@ import com.google.common.primitives.Longs;
|
|||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.aggregation;
|
||||
|
||||
|
||||
import com.google.common.primitives.Floats;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.druid.processing.FloatMetricSelector;
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class HistogramVisual
|
||||
|
|
|
@ -28,7 +28,6 @@ import com.metamx.common.IAE;
|
|||
import com.metamx.druid.processing.FloatMetricSelector;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.ContextAction;
|
||||
import org.mozilla.javascript.ContextFactory;
|
||||
|
@ -228,7 +227,6 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
|
|||
final Function fnCombine = getScriptFunction("combine", scope);
|
||||
Context.exit();
|
||||
|
||||
|
||||
return new JavaScriptAggregator.ScriptAggregator()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.primitives.Doubles;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.primitives.Doubles;
|
||||
import com.metamx.druid.processing.MetricSelectorFactory;
|
||||
|
||||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.IAE;
|
||||
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
|
|
@ -23,8 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.ISE;
|
||||
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.aggregation.post;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.db;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.skife.config.Config;
|
||||
|
||||
|
|
|
@ -27,7 +27,6 @@ import com.google.common.collect.Lists;
|
|||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.exception.FormattedException;
|
||||
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package com.metamx.druid.input;
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
|
|
|
@ -19,11 +19,6 @@
|
|||
|
||||
package com.metamx.druid.jackson;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||
|
|
|
@ -25,10 +25,6 @@ import com.fasterxml.jackson.databind.SerializerProvider;
|
|||
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer;
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
|
|
|
@ -43,20 +43,6 @@ import com.metamx.common.Granularity;
|
|||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.skife.jdbi.v2.sqlobject.customizers.Mapper;
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package com.metamx.druid.jackson;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.core.JsonToken;
|
||||
|
|
|
@ -29,7 +29,6 @@ import javax.management.NotCompliantMBeanException;
|
|||
import javax.management.ObjectName;
|
||||
import java.lang.management.ManagementFactory;
|
||||
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LogLevelAdjuster implements LogLevelAdjusterMBean
|
||||
|
|
|
@ -10,7 +10,6 @@ public class LinearPartitionChunk <T> implements PartitionChunk<T>
|
|||
return new LinearPartitionChunk<T>(chunkNumber, obj);
|
||||
}
|
||||
|
||||
|
||||
public LinearPartitionChunk(
|
||||
int chunkNumber,
|
||||
T object
|
||||
|
|
|
@ -39,7 +39,6 @@ public class StringPartitionChunk<T> implements PartitionChunk<T>
|
|||
return new StringPartitionChunk<T>(start, end, chunkNumber, obj);
|
||||
}
|
||||
|
||||
|
||||
public StringPartitionChunk(
|
||||
String start,
|
||||
String end,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue