mirror of
https://github.com/apache/druid.git
synced 2025-03-04 16:29:17 +00:00
Initial commit of OSS Druid Code
This commit is contained in:
parent
70e993806e
commit
9d41599967
16
.gitignore
vendored
Normal file
16
.gitignore
vendored
Normal file
@ -0,0 +1,16 @@
|
||||
dist
|
||||
target
|
||||
*.iml
|
||||
*.ipr
|
||||
*.iws
|
||||
*.tar.gz
|
||||
*.swp
|
||||
*.swo
|
||||
.classpath
|
||||
.idea
|
||||
.project
|
||||
.settings/org.eclipse.jdt.core.prefs
|
||||
.settings/org.maven.ide.eclipse.prefs
|
||||
client/.settings/org.eclipse.jdt.core.prefs
|
||||
common/.settings/org.eclipse.jdt.core.prefs
|
||||
server/.settings/org.eclipse.jdt.core.prefs
|
181
build.gradle
Normal file
181
build.gradle
Normal file
@ -0,0 +1,181 @@
|
||||
allprojects {
|
||||
apply plugin: 'java'
|
||||
apply plugin: 'idea'
|
||||
|
||||
group="com.metamx.druid"
|
||||
version="2.5.60"
|
||||
status="release"
|
||||
|
||||
repositories {
|
||||
mavenCentral()
|
||||
maven {
|
||||
url "https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local"
|
||||
}
|
||||
maven {
|
||||
url "http://repo.codahale.com"
|
||||
}
|
||||
maven {
|
||||
url "http://nativelibs4java.sourceforge.net/maven"
|
||||
}
|
||||
maven {
|
||||
url "https://repository.jboss.org/nexus/content/repositories/thirdparty-uploads"
|
||||
}
|
||||
maven {
|
||||
url "https://metamx.artifactoryonline.com/metamx/libs-releases-local"
|
||||
}
|
||||
add(new org.apache.ivy.plugins.resolver.URLResolver()) {
|
||||
name = "nfrepo-${project.status}s"
|
||||
def artifactsUrl = artifactsBase + name
|
||||
addIvyPattern "${artifactsUrl}/[organisation]/[module]/[revision]/[module]-[revision]-ivy.[ext]"
|
||||
addArtifactPattern "${artifactsUrl}/[organisation]/[module]/[revision]/[artifact]-[revision](-[classifier]).[ext]"
|
||||
m2compatible = true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
project(':common') {
|
||||
dependencies {
|
||||
compile 'com.metamx:java-util:0.15.0'
|
||||
compile 'joda-time:joda-time:2.1'
|
||||
compile 'com.google.guava:guava:11.0.1'
|
||||
compile 'log4j:log4j:1.2.16'
|
||||
compile 'commons-codec:commons-codec:1.3'
|
||||
compile 'org.codehaus.jackson:jackson-core-asl:1.9.9'
|
||||
compile 'it.uniroma3.mat:extendedset:1.3.2'
|
||||
compile 'mysql:mysql-connector-java:5.1.18'
|
||||
compile 'org.jdbi:jdbi:2.32'
|
||||
compile 'commons-pool:commons-pool:1.6'
|
||||
compile 'commons-dbcp:commons-dbcp:1.4'
|
||||
compile 'org.mozilla:rhino:1.7R4'
|
||||
compile 'com.netflix.curator:curator-x-discovery:1.0.24e'
|
||||
compile 'com.netflix.curator:curator-recipes:1.0.16'
|
||||
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
||||
|
||||
project(':client') {
|
||||
dependencies {
|
||||
compile project(':common')
|
||||
compile 'com.metamx:http-client:0.6.1'
|
||||
compile 'com.metamx:emitter:0.0.7'
|
||||
compile 'org.mortbay.jetty:jetty:6.1.26'
|
||||
compile 'joda-time:joda-time:2.1'
|
||||
compile 'com.google.inject:guice:3.0'
|
||||
compile 'com.google.guava:guava:11.0.1'
|
||||
compile 'com.sun.jersey:jersey-server:1.9.1'
|
||||
compile 'com.sun.jersey:jersey-core:1.9.1'
|
||||
compile 'com.sun.jersey.contribs:jersey-guice:1.9.1'
|
||||
compile 'com.google.inject.extensions:guice-servlet:3.0'
|
||||
compile 'org.codehaus.jackson:jackson-jaxrs:1.9.9'
|
||||
compile 'org.codehaus.jackson:jackson-smile:1.9.9'
|
||||
compile 'log4j:log4j:1.2.16'
|
||||
compile 'org.slf4j:slf4j-log4j12:1.6.2'
|
||||
compile 'com.github.sgroschupf:zkclient:0.1'
|
||||
compile 'commons-codec:commons-codec:1.3'
|
||||
compile 'org.skife.config:config-magic:0.9'
|
||||
compile 'com.metamx:server-metrics:0.0.2'
|
||||
compile 'com.davekoelle:alphanum:1.0.3'
|
||||
compile 'com.ibm.icu:icu4j:4.8.1'
|
||||
compile 'org.apache.zookeeper:zookeeper:3.3.5'
|
||||
compile 'com.netflix.curator:curator-recipes:1.0.16'
|
||||
compile 'com.netflix.curator:curator-framework:1.0.24e'
|
||||
compile 'com.netflix.curator:curator-client:1.0.24e'
|
||||
compile 'com.netflix.curator:curator-x-discovery:1.0.24e'
|
||||
compile 'com.netflix.curator:curator-x-zkclient-bridge:1.0.24e'
|
||||
compile 'netflix:nfzookeeper:1.34.0'
|
||||
compile 'netflix:nflibrary:2.12'
|
||||
compile 'netflix:platform-management:2.12'
|
||||
compile 'commons-configuration:commons-configuration:1.6'
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
||||
|
||||
project(':index-common') {
|
||||
dependencies {
|
||||
compile project(':common')
|
||||
compile 'com.ning:compress-lzf:0.8.4'
|
||||
compile 'net.java.dev.jets3t:jets3t:0.8.1'
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
}
|
||||
}
|
||||
|
||||
project(':indexer') {
|
||||
dependencies {
|
||||
compile project(':index-common')
|
||||
compile project(':server')
|
||||
compile 'org.apache.hadoop:hadoop-core:0.20.2'
|
||||
compile('com.amazonaws:aws-java-sdk:1.3.11') {
|
||||
exclude group: 'javax.mail', module: 'mail'
|
||||
}
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
}
|
||||
}
|
||||
|
||||
project(':server') {
|
||||
dependencies {
|
||||
compile project(':client')
|
||||
compile project(':index-common')
|
||||
compile 'org.mortbay.jetty:jetty:6.1.26'
|
||||
compile 'commons-lang:commons-lang:2.6'
|
||||
compile 'commons-io:commons-io:2.0.1'
|
||||
compile 'com.google.inject:guice:3.0'
|
||||
compile 'com.google.guava:guava:11.0.1'
|
||||
compile 'log4j:log4j:1.2.16'
|
||||
compile 'org.slf4j:slf4j-log4j12:1.6.2'
|
||||
compile 'commons-cli:commons-cli:1.2'
|
||||
compile 'com.github.sgroschupf:zkclient:0.1'
|
||||
compile 'com.ning:compress-lzf:0.8.4'
|
||||
compile 'com.sun.jersey:jersey-server:1.9.1'
|
||||
compile 'com.sun.jersey:jersey-core:1.9.1'
|
||||
compile 'com.sun.jersey.contribs:jersey-guice:1.9.1'
|
||||
compile 'com.google.inject.extensions:guice-servlet:3.0'
|
||||
compile 'org.codehaus.jackson:jackson-jaxrs:1.9.9'
|
||||
compile 'commons-codec:commons-codec:1.3'
|
||||
compile 'commons-logging:commons-logging:1.1.1'
|
||||
compile 'commons-httpclient:commons-httpclient:3.1'
|
||||
compile 'com.jamesmurty.utils:java-xmlbuilder:0.4'
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
||||
|
||||
project(':realtime') {
|
||||
dependencies {
|
||||
compile project(':server')
|
||||
compile project(':client')
|
||||
compile 'org.scala-lang:scala-library:2.9.1'
|
||||
compile 'org.scala-lang:scala-compiler:2.9.1'
|
||||
compile 'kafka:core-kafka:0.6-mmx11'
|
||||
compile 'org.apache.directory.studio:org.apache.commons.collections:3.2.1'
|
||||
compile 'com.codahale:jerkson_2.9.1:0.5.0'
|
||||
compile 'commons-logging:commons-logging:1.1.1'
|
||||
compile 'commons-httpclient:commons-httpclient:3.1'
|
||||
compile 'com.jamesmurty.utils:java-xmlbuilder:0.4'
|
||||
compile group: "kafka-core", name: "kafka-core", version: "0.7.1", configuration: "compile"
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
||||
|
||||
project(':merger') {
|
||||
dependencies {
|
||||
compile project(':server')
|
||||
compile project(':client')
|
||||
compile project(':indexer')
|
||||
compile project(':realtime')
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
||||
|
||||
project(':integration-testing') {
|
||||
dependencies {
|
||||
compile project(':client')
|
||||
compile project(':common')
|
||||
testCompile 'junit:junit:4.8.1'
|
||||
testCompile 'org.easymock:easymock:3.0'
|
||||
}
|
||||
}
|
36
build.sh
Executable file
36
build.sh
Executable file
@ -0,0 +1,36 @@
|
||||
#!/bin/bash -e
|
||||
|
||||
PROJECT=druid
|
||||
|
||||
DIST_DIR=dist/tar
|
||||
|
||||
SCRIPT_DIR=`dirname $0`
|
||||
pushd $SCRIPT_DIR
|
||||
SCRIPT_DIR=`pwd`
|
||||
popd
|
||||
|
||||
VERSION=`cat pom.xml | grep version | head -2 | tail -1 | sed 's_.*<version>\([^<]*\)</version>.*_\1_'`
|
||||
TAR_FILE=${SCRIPT_DIR}/${PROJECT}-${VERSION}.tar.gz
|
||||
|
||||
echo Using Version[${VERSION}] and creating zip file ${TAR_FILE}
|
||||
|
||||
rm -f ${TAR_FILE}
|
||||
mvn clean
|
||||
mvn package
|
||||
|
||||
if [ $? -ne "0" ]; then
|
||||
echo "mvn package failed"
|
||||
exit 2;
|
||||
fi
|
||||
|
||||
rm -rf ${DIST_DIR}
|
||||
mkdir -p ${DIST_DIR}/lib
|
||||
|
||||
cp binary-artifact/target/${PROJECT}-binary-artifact-${VERSION}-selfcontained.jar ${DIST_DIR}/lib/
|
||||
cp -r bin ${DIST_DIR}/ # (bin/ is provided by java-shell)
|
||||
|
||||
cd ${DIST_DIR}
|
||||
tar czf ${TAR_FILE} *
|
||||
echo
|
||||
echo Created ${TAR_FILE}:
|
||||
tar tf ${TAR_FILE} | sed -r 's/^/ /'
|
207
client/pom.xml
Normal file
207
client/pom.xml
Normal file
@ -0,0 +1,207 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-client</artifactId>
|
||||
<version>0.1.0-SNAPSHOT</version>
|
||||
<name>druid-client</name>
|
||||
<description>druid-client</description>
|
||||
<scm>
|
||||
<connection>scm:git:ssh://git@github.com/metamx/druid.git</connection>
|
||||
<developerConnection>scm:git:ssh://git@github.com/metamx/druid.git</developerConnection>
|
||||
<url>http://www.github.com/metamx/druid</url>
|
||||
</scm>
|
||||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
</properties>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-index-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>http-client</artifactId>
|
||||
<version>0.6.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>emitter</artifactId>
|
||||
<version>0.0.7</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.mortbay.jetty</groupId>
|
||||
<artifactId>jetty</artifactId>
|
||||
<version>6.1.26</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
<version>3.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<version>11.0.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-server</artifactId>
|
||||
<version>1.9.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-core</artifactId>
|
||||
<version>1.9.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey.contribs</groupId>
|
||||
<artifactId>jersey-guice</artifactId>
|
||||
<version>1.9.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-servlet</artifactId>
|
||||
<version>3.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-jaxrs</artifactId>
|
||||
<version>1.9.9</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-smile</artifactId>
|
||||
<version>1.9.9</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
<version>1.2.16</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
<version>1.6.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.sgroschupf</groupId>
|
||||
<artifactId>zkclient</artifactId>
|
||||
<version>0.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-codec</groupId>
|
||||
<artifactId>commons-codec</artifactId>
|
||||
<version>1.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.skife.config</groupId>
|
||||
<artifactId>config-magic</artifactId>
|
||||
<version>0.9</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>server-metrics</artifactId>
|
||||
<version>0.0.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.davekoelle</groupId>
|
||||
<artifactId>alphanum</artifactId>
|
||||
<version>1.0.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.ibm.icu</groupId>
|
||||
<artifactId>icu4j</artifactId>
|
||||
<version>4.8.1</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<version>4.8.1</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<version>3.0</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<version>2.3.2</version>
|
||||
<configuration>
|
||||
<source>1.6</source>
|
||||
<target>1.6</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-shade-plugin</artifactId>
|
||||
<version>1.6</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>shade</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputFile>
|
||||
${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar
|
||||
</outputFile>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-help-plugin</artifactId>
|
||||
<version>2.1.1</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<version>2.7.2</version>
|
||||
<configuration>
|
||||
<systemPropertyVariables>
|
||||
<user.timezone>UTC</user.timezone>
|
||||
</systemPropertyVariables>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<version>2.4</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
111
client/src/main/java/com/metamx/druid/BaseQuery.java
Normal file
111
client/src/main/java/com/metamx/druid/BaseQuery.java
Normal file
@ -0,0 +1,111 @@
|
||||
package com.metamx.druid;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class BaseQuery<T> implements Query<T>
|
||||
{
|
||||
private final String dataSource;
|
||||
private final Map<String, String> context;
|
||||
private final QuerySegmentSpec querySegmentSpec;
|
||||
|
||||
private volatile Duration duration;
|
||||
|
||||
public BaseQuery(
|
||||
String dataSource,
|
||||
QuerySegmentSpec querySegmentSpec,
|
||||
Map<String, String> context
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(dataSource, "dataSource can't be null");
|
||||
Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null");
|
||||
|
||||
this.dataSource = dataSource.toLowerCase();
|
||||
this.context = context;
|
||||
this.querySegmentSpec = querySegmentSpec;
|
||||
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty("intervals")
|
||||
public QuerySegmentSpec getQuerySegmentSpec()
|
||||
{
|
||||
return querySegmentSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(QuerySegmentWalker walker)
|
||||
{
|
||||
return querySegmentSpec.lookup(this, walker).run(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Interval> getIntervals()
|
||||
{
|
||||
return querySegmentSpec.getIntervals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getDuration()
|
||||
{
|
||||
if (duration == null) {
|
||||
Duration totalDuration = new Duration(0);
|
||||
for (Interval interval : querySegmentSpec.getIntervals()) {
|
||||
if (interval != null) {
|
||||
totalDuration = totalDuration.plus(interval.toDuration());
|
||||
}
|
||||
}
|
||||
duration = totalDuration;
|
||||
}
|
||||
|
||||
return duration;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, String> getContext()
|
||||
{
|
||||
return context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getContextValue(String key)
|
||||
{
|
||||
return context == null ? null : context.get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getContextValue(String key, String defaultValue)
|
||||
{
|
||||
String retVal = getContextValue(key);
|
||||
return retVal == null ? defaultValue : retVal;
|
||||
}
|
||||
|
||||
protected Map<String, String> computeOverridenContext(Map<String, String> overrides)
|
||||
{
|
||||
Map<String, String> overridden = Maps.newTreeMap();
|
||||
final Map<String, String> context = getContext();
|
||||
if (context != null) {
|
||||
overridden.putAll(context);
|
||||
}
|
||||
overridden.putAll(overrides);
|
||||
|
||||
return overridden;
|
||||
}
|
||||
}
|
594
client/src/main/java/com/metamx/druid/Druids.java
Normal file
594
client/src/main/java/com/metamx/druid/Druids.java
Normal file
@ -0,0 +1,594 @@
|
||||
package com.metamx.druid;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.druid.query.filter.AndDimFilter;
|
||||
import com.metamx.druid.query.filter.DimFilter;
|
||||
import com.metamx.druid.query.filter.NoopDimFilter;
|
||||
import com.metamx.druid.query.filter.NotDimFilter;
|
||||
import com.metamx.druid.query.filter.OrDimFilter;
|
||||
import com.metamx.druid.query.filter.SelectorDimFilter;
|
||||
import com.metamx.druid.query.search.InsensitiveContainsSearchQuerySpec;
|
||||
import com.metamx.druid.query.search.SearchQuery;
|
||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
import com.metamx.druid.result.Result;
|
||||
import com.metamx.druid.result.SearchResultValue;
|
||||
import com.metamx.druid.result.TimeBoundaryResultValue;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class Druids
|
||||
{
|
||||
private Druids()
|
||||
{
|
||||
throw new AssertionError();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for AndDimFilter.
|
||||
* <p/>
|
||||
* Required: fields() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* AndDimFilter andDimFilter = Druids.newAndDimFilterBuilder()
|
||||
* .fields(listOfDimFilterFields)
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see AndDimFilter
|
||||
*/
|
||||
public static class AndDimFilterBuilder
|
||||
{
|
||||
private List<DimFilter> fields;
|
||||
|
||||
public AndDimFilterBuilder()
|
||||
{
|
||||
fields = Lists.newArrayList();
|
||||
}
|
||||
|
||||
public AndDimFilter build()
|
||||
{
|
||||
return new AndDimFilter(fields);
|
||||
}
|
||||
|
||||
public AndDimFilterBuilder copy(AndDimFilterBuilder builder)
|
||||
{
|
||||
return new AndDimFilterBuilder()
|
||||
.fields(builder.fields);
|
||||
}
|
||||
|
||||
public AndDimFilterBuilder fields(List<DimFilter> f)
|
||||
{
|
||||
fields.addAll(f);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static AndDimFilterBuilder newAndDimFilterBuilder()
|
||||
{
|
||||
return new AndDimFilterBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for OrDimFilter.
|
||||
* <p/>
|
||||
* Required: fields() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* OrDimFilter orDimFilter = Druids.newOrDimFilterBuilder()
|
||||
* .fields(listOfDimFilterFields)
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see OrDimFilter
|
||||
*/
|
||||
public static class OrDimFilterBuilder
|
||||
{
|
||||
private List<DimFilter> fields;
|
||||
|
||||
public OrDimFilterBuilder()
|
||||
{
|
||||
fields = Lists.newArrayList();
|
||||
}
|
||||
|
||||
public OrDimFilter build()
|
||||
{
|
||||
return new OrDimFilter(fields);
|
||||
}
|
||||
|
||||
public OrDimFilterBuilder copy(OrDimFilterBuilder builder)
|
||||
{
|
||||
return new OrDimFilterBuilder()
|
||||
.fields(builder.fields);
|
||||
}
|
||||
|
||||
public OrDimFilterBuilder fields(String dimensionName, String value, String... values)
|
||||
{
|
||||
fields = Lists.<DimFilter>newArrayList(new SelectorDimFilter(dimensionName, value));
|
||||
for (String val : values) {
|
||||
fields.add(new SelectorDimFilter(dimensionName, val));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public OrDimFilterBuilder fields(List<DimFilter> f)
|
||||
{
|
||||
fields.addAll(f);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static OrDimFilterBuilder newOrDimFilterBuilder()
|
||||
{
|
||||
return new OrDimFilterBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for NotDimFilter.
|
||||
* <p/>
|
||||
* Required: field() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* NotDimFilter notDimFilter = Druids.newNotDimFilterBuilder()
|
||||
* .field(dimFilterField)
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see NotDimFilter
|
||||
*/
|
||||
public static class NotDimFilterBuilder
|
||||
{
|
||||
private DimFilter field;
|
||||
|
||||
public NotDimFilterBuilder()
|
||||
{
|
||||
field = null;
|
||||
}
|
||||
|
||||
public NotDimFilter build()
|
||||
{
|
||||
return new NotDimFilter(field);
|
||||
}
|
||||
|
||||
public NotDimFilterBuilder copy(NotDimFilterBuilder builder)
|
||||
{
|
||||
return new NotDimFilterBuilder()
|
||||
.field(builder.field);
|
||||
}
|
||||
|
||||
public NotDimFilterBuilder field(DimFilter f)
|
||||
{
|
||||
field = f;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static NotDimFilterBuilder newNotDimFilterBuilder()
|
||||
{
|
||||
return new NotDimFilterBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for SelectorDimFilter.
|
||||
* <p/>
|
||||
* Required: dimension() and value() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* Selector selDimFilter = Druids.newSelectorDimFilterBuilder()
|
||||
* .dimension("test")
|
||||
* .value("sample")
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see SelectorDimFilter
|
||||
*/
|
||||
public static class SelectorDimFilterBuilder
|
||||
{
|
||||
private String dimension;
|
||||
private String value;
|
||||
|
||||
public SelectorDimFilterBuilder()
|
||||
{
|
||||
dimension = "";
|
||||
value = "";
|
||||
}
|
||||
|
||||
public SelectorDimFilter build()
|
||||
{
|
||||
return new SelectorDimFilter(dimension, value);
|
||||
}
|
||||
|
||||
public SelectorDimFilterBuilder copy(SelectorDimFilterBuilder builder)
|
||||
{
|
||||
return new SelectorDimFilterBuilder()
|
||||
.dimension(builder.dimension)
|
||||
.value(builder.value);
|
||||
}
|
||||
|
||||
public SelectorDimFilterBuilder dimension(String d)
|
||||
{
|
||||
dimension = d;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SelectorDimFilterBuilder value(String v)
|
||||
{
|
||||
value = v;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static SelectorDimFilterBuilder newSelectorDimFilterBuilder()
|
||||
{
|
||||
return new SelectorDimFilterBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for NoopDimFilter.
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* NoopDimFilter noopDimFilter = Druids.newNoopDimFilterBuilder()
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see NotDimFilter
|
||||
*/
|
||||
public static class NoopDimFilterBuilder
|
||||
{
|
||||
public NoopDimFilter build()
|
||||
{
|
||||
return new NoopDimFilter();
|
||||
}
|
||||
}
|
||||
|
||||
public static NoopDimFilterBuilder newNoopDimFilterBuilder()
|
||||
{
|
||||
return new NoopDimFilterBuilder();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A Builder for SearchQuery.
|
||||
* <p/>
|
||||
* Required: dataSource(), intervals(), dimensions() and query() must be called before build()
|
||||
* <p/>
|
||||
* Optional: filters(), granularity(), and context() can be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* SearchQuery query = Druids.newSearchQueryBuilder()
|
||||
* .dataSource("Example")
|
||||
* .dimensions(listofEgDims)
|
||||
* .query(exampleQuery)
|
||||
* .intervals("2012-01-01/2012-01-02")
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see com.metamx.druid.query.search.SearchQuery
|
||||
*/
|
||||
public static class SearchQueryBuilder
|
||||
{
|
||||
private String dataSource;
|
||||
private DimFilter dimFilter;
|
||||
private QueryGranularity granularity;
|
||||
private int limit;
|
||||
private QuerySegmentSpec querySegmentSpec;
|
||||
private List<String> dimensions;
|
||||
private SearchQuerySpec querySpec;
|
||||
private Map<String, String> context;
|
||||
|
||||
public SearchQueryBuilder()
|
||||
{
|
||||
dataSource = "";
|
||||
dimFilter = null;
|
||||
granularity = QueryGranularity.ALL;
|
||||
limit = 0;
|
||||
querySegmentSpec = null;
|
||||
dimensions = null;
|
||||
querySpec = null;
|
||||
context = null;
|
||||
}
|
||||
|
||||
public SearchQuery build()
|
||||
{
|
||||
return new SearchQuery(
|
||||
dataSource,
|
||||
dimFilter,
|
||||
granularity,
|
||||
limit,
|
||||
querySegmentSpec,
|
||||
dimensions,
|
||||
querySpec,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
||||
public SearchQueryBuilder copy(SearchQuery query)
|
||||
{
|
||||
return new SearchQueryBuilder()
|
||||
.dataSource(query.getDataSource())
|
||||
.intervals(query.getQuerySegmentSpec())
|
||||
.filters(query.getDimensionsFilter())
|
||||
.granularity(query.getGranularity())
|
||||
.limit(query.getLimit())
|
||||
.dimensions(query.getDimensions())
|
||||
.query(query.getQuery())
|
||||
.context(query.getContext());
|
||||
}
|
||||
|
||||
public SearchQueryBuilder copy(SearchQueryBuilder builder)
|
||||
{
|
||||
return new SearchQueryBuilder()
|
||||
.dataSource(builder.dataSource)
|
||||
.intervals(builder.querySegmentSpec)
|
||||
.filters(builder.dimFilter)
|
||||
.granularity(builder.granularity)
|
||||
.limit(builder.limit)
|
||||
.dimensions(builder.dimensions)
|
||||
.query(builder.querySpec)
|
||||
.context(builder.context);
|
||||
}
|
||||
|
||||
public SearchQueryBuilder dataSource(String d)
|
||||
{
|
||||
dataSource = d;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder filters(String dimensionName, String value)
|
||||
{
|
||||
dimFilter = new SelectorDimFilter(dimensionName, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder filters(String dimensionName, String value, String... values)
|
||||
{
|
||||
List<DimFilter> fields = Lists.<DimFilter>newArrayList(new SelectorDimFilter(dimensionName, value));
|
||||
for (String val : values) {
|
||||
fields.add(new SelectorDimFilter(dimensionName, val));
|
||||
}
|
||||
dimFilter = new OrDimFilter(fields);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder filters(DimFilter f)
|
||||
{
|
||||
dimFilter = f;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder granularity(String g)
|
||||
{
|
||||
granularity = QueryGranularity.fromString(g);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder granularity(QueryGranularity g)
|
||||
{
|
||||
granularity = g;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder limit(int l)
|
||||
{
|
||||
limit = l;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder intervals(QuerySegmentSpec q)
|
||||
{
|
||||
querySegmentSpec = q;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder intervals(String s)
|
||||
{
|
||||
querySegmentSpec = new LegacySegmentSpec(s);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder intervals(List<Interval> l)
|
||||
{
|
||||
querySegmentSpec = new LegacySegmentSpec(l);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder dimensions(String d)
|
||||
{
|
||||
dimensions = Lists.newArrayList(d);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder dimensions(List<String> d)
|
||||
{
|
||||
dimensions = d;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder query(SearchQuerySpec s)
|
||||
{
|
||||
querySpec = s;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder query(String q)
|
||||
{
|
||||
querySpec = new InsensitiveContainsSearchQuerySpec(q, null);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder query(Map<String, Object> q)
|
||||
{
|
||||
querySpec = new InsensitiveContainsSearchQuerySpec((String) q.get("value"), null);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SearchQueryBuilder context(Map<String, String> c)
|
||||
{
|
||||
context = c;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static SearchQueryBuilder newSearchQueryBuilder()
|
||||
{
|
||||
return new SearchQueryBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for TimeBoundaryQuery.
|
||||
* <p/>
|
||||
* Required: dataSource() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* TimeBoundaryQuery query = new MaxTimeQueryBuilder()
|
||||
* .dataSource("Example")
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see com.metamx.druid.query.timeboundary.TimeBoundaryQuery
|
||||
*/
|
||||
public static class TimeBoundaryQueryBuilder
|
||||
{
|
||||
private String dataSource;
|
||||
private QuerySegmentSpec querySegmentSpec;
|
||||
private Map<String, String> context;
|
||||
|
||||
public TimeBoundaryQueryBuilder()
|
||||
{
|
||||
dataSource = "";
|
||||
querySegmentSpec = null;
|
||||
context = null;
|
||||
}
|
||||
|
||||
public TimeBoundaryQuery build()
|
||||
{
|
||||
return new TimeBoundaryQuery(
|
||||
dataSource,
|
||||
querySegmentSpec,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder copy(TimeBoundaryQueryBuilder builder)
|
||||
{
|
||||
return new TimeBoundaryQueryBuilder()
|
||||
.dataSource(builder.dataSource)
|
||||
.intervals(builder.querySegmentSpec)
|
||||
.context(builder.context);
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder dataSource(String d)
|
||||
{
|
||||
dataSource = d;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder intervals(QuerySegmentSpec q)
|
||||
{
|
||||
querySegmentSpec = q;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder intervals(String s)
|
||||
{
|
||||
querySegmentSpec = new LegacySegmentSpec(s);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder intervals(List<Interval> l)
|
||||
{
|
||||
querySegmentSpec = new LegacySegmentSpec(l);
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeBoundaryQueryBuilder context(Map<String, String> c)
|
||||
{
|
||||
context = c;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static TimeBoundaryQueryBuilder newTimeBoundaryQueryBuilder()
|
||||
{
|
||||
return new TimeBoundaryQueryBuilder();
|
||||
}
|
||||
|
||||
/**
|
||||
* A Builder for Result.
|
||||
* <p/>
|
||||
* Required: timestamp() and value() must be called before build()
|
||||
* <p/>
|
||||
* Usage example:
|
||||
* <pre><code>
|
||||
* Result<T> result = Druids.newResultBuilder()
|
||||
* .timestamp(egDateTime)
|
||||
* .value(egValue)
|
||||
* .build();
|
||||
* </code></pre>
|
||||
*
|
||||
* @see com.metamx.druid.result.Result
|
||||
*/
|
||||
public static class ResultBuilder<T>
|
||||
{
|
||||
private DateTime timestamp;
|
||||
private Object value;
|
||||
|
||||
public ResultBuilder()
|
||||
{
|
||||
timestamp = new DateTime(0);
|
||||
value = null;
|
||||
}
|
||||
|
||||
public Result<T> build()
|
||||
{
|
||||
return new Result<T>(timestamp, (T) value);
|
||||
}
|
||||
|
||||
public ResultBuilder copy(ResultBuilder builder)
|
||||
{
|
||||
return new ResultBuilder()
|
||||
.timestamp(builder.timestamp)
|
||||
.value(builder.value);
|
||||
}
|
||||
|
||||
public ResultBuilder<T> timestamp(DateTime t)
|
||||
{
|
||||
timestamp = t;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ResultBuilder<T> value(Object v)
|
||||
{
|
||||
value = v;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static ResultBuilder newResultBuilder()
|
||||
{
|
||||
return new ResultBuilder();
|
||||
}
|
||||
|
||||
public static ResultBuilder<SearchResultValue> newSearchResultBuilder()
|
||||
{
|
||||
return new ResultBuilder<SearchResultValue>();
|
||||
}
|
||||
|
||||
public static ResultBuilder<TimeBoundaryResultValue> newTimeBoundaryResultBuilder()
|
||||
{
|
||||
return new ResultBuilder<TimeBoundaryResultValue>();
|
||||
}
|
||||
}
|
47
client/src/main/java/com/metamx/druid/Query.java
Normal file
47
client/src/main/java/com/metamx/druid/Query.java
Normal file
@ -0,0 +1,47 @@
|
||||
package com.metamx.druid;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.query.group.GroupByQuery;
|
||||
import com.metamx.druid.query.search.SearchQuery;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
|
||||
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
|
||||
@JsonSubTypes.Type(name = "groupBy", value= GroupByQuery.class)
|
||||
})
|
||||
public interface Query<T>
|
||||
{
|
||||
public static final String SEARCH = "search";
|
||||
public static final String TIME_BOUNDARY = "timeBoundary";
|
||||
|
||||
public String getDataSource();
|
||||
|
||||
public boolean hasFilters();
|
||||
|
||||
public String getType();
|
||||
|
||||
public Sequence<T> run(QuerySegmentWalker walker);
|
||||
|
||||
public List<Interval> getIntervals();
|
||||
|
||||
public Duration getDuration();
|
||||
|
||||
public String getContextValue(String key);
|
||||
|
||||
public String getContextValue(String key, String defaultValue);
|
||||
|
||||
public Query<T> withOverriddenContext(Map<String, String> contextOverride);
|
||||
|
||||
public Query<T> withQuerySegmentSpec(QuerySegmentSpec spec);
|
||||
}
|
@ -0,0 +1,27 @@
|
||||
package com.metamx.druid;
|
||||
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.metamx.druid.result.Result;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ResultGranularTimestampComparator<T> implements Comparator<Result<T>>
|
||||
{
|
||||
private final QueryGranularity gran;
|
||||
|
||||
public ResultGranularTimestampComparator(QueryGranularity granularity)
|
||||
{
|
||||
this.gran = granularity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(Result<T> r1, Result<T> r2)
|
||||
{
|
||||
return Longs.compare(
|
||||
gran.truncate(r1.getTimestamp().getMillis()),
|
||||
gran.truncate(r2.getTimestamp().getMillis())
|
||||
);
|
||||
}
|
||||
}
|
55
client/src/main/java/com/metamx/druid/SearchBinaryFn.java
Normal file
55
client/src/main/java/com/metamx/druid/SearchBinaryFn.java
Normal file
@ -0,0 +1,55 @@
|
||||
package com.metamx.druid;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.druid.query.search.SearchHit;
|
||||
import com.metamx.druid.query.search.SearchSortSpec;
|
||||
import com.metamx.druid.result.Result;
|
||||
import com.metamx.druid.result.SearchResultValue;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SearchBinaryFn implements BinaryFn<Result<SearchResultValue>, Result<SearchResultValue>, Result<SearchResultValue>>
|
||||
{
|
||||
private final SearchSortSpec searchSortSpec;
|
||||
private final QueryGranularity gran;
|
||||
|
||||
public SearchBinaryFn(
|
||||
SearchSortSpec searchSortSpec,
|
||||
QueryGranularity granularity
|
||||
)
|
||||
{
|
||||
this.searchSortSpec = searchSortSpec;
|
||||
this.gran = granularity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result<SearchResultValue> apply(Result<SearchResultValue> arg1, Result<SearchResultValue> arg2)
|
||||
{
|
||||
if (arg1 == null) {
|
||||
return arg2;
|
||||
}
|
||||
|
||||
if (arg2 == null) {
|
||||
return arg1;
|
||||
}
|
||||
|
||||
SearchResultValue arg1Vals = arg1.getValue();
|
||||
SearchResultValue arg2Vals = arg2.getValue();
|
||||
|
||||
TreeSet<SearchHit> results = Sets.newTreeSet(searchSortSpec.getComparator());
|
||||
results.addAll(Lists.newArrayList(arg1Vals));
|
||||
results.addAll(Lists.newArrayList(arg2Vals));
|
||||
|
||||
return (gran instanceof AllGranularity)
|
||||
? new Result<SearchResultValue>(arg1.getTimestamp(), new SearchResultValue(Lists.newArrayList(results)))
|
||||
: new Result<SearchResultValue>(
|
||||
gran.toDateTime(gran.truncate(arg1.getTimestamp().getMillis())),
|
||||
new SearchResultValue(Lists.newArrayList(results))
|
||||
);
|
||||
}
|
||||
}
|
@ -0,0 +1,345 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.LazySequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.TimelineObjectHolder;
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.cache.Cache;
|
||||
import com.metamx.druid.client.cache.CacheBroker;
|
||||
import com.metamx.druid.client.selector.ServerSelector;
|
||||
import com.metamx.druid.partition.PartitionChunk;
|
||||
import com.metamx.druid.query.CacheStrategy;
|
||||
import com.metamx.druid.query.MetricManipulationFn;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChest;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||
import com.metamx.druid.result.Result;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||
{
|
||||
private static final Logger log = new Logger(CachingClusteredClient.class);
|
||||
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final ServerView serverView;
|
||||
private final CacheBroker cacheBroker;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public CachingClusteredClient(
|
||||
QueryToolChestWarehouse warehouse,
|
||||
ServerView serverView,
|
||||
CacheBroker cacheBroker,
|
||||
ObjectMapper objectMapper
|
||||
)
|
||||
{
|
||||
this.warehouse = warehouse;
|
||||
this.serverView = serverView;
|
||||
this.cacheBroker = cacheBroker;
|
||||
this.objectMapper = objectMapper;
|
||||
|
||||
serverView.registerSegmentCallback(
|
||||
Executors.newFixedThreadPool(
|
||||
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CCClient-ServerView-CB-%d").build()
|
||||
),
|
||||
new ServerView.BaseSegmentCallback()
|
||||
{
|
||||
@Override
|
||||
public ServerView.CallbackAction segmentRemoved(DruidServer server, DataSegment segment)
|
||||
{
|
||||
CachingClusteredClient.this.cacheBroker.provideCache(segment.getIdentifier()).close();
|
||||
return ServerView.CallbackAction.CONTINUE;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
||||
final CacheStrategy<T, Query<T>> strategy = toolChest.getCacheStrategy(query);
|
||||
|
||||
final Map<DruidServer, List<SegmentDescriptor>> segs = Maps.newTreeMap();
|
||||
final List<Pair<DateTime, byte[]>> cachedResults = Lists.newArrayList();
|
||||
final Map<String, CachePopulator> cachePopulatorMap = Maps.newHashMap();
|
||||
|
||||
final boolean useCache = Boolean.parseBoolean(query.getContextValue("useCache", "true")) && strategy != null;
|
||||
final boolean populateCache = Boolean.parseBoolean(query.getContextValue("populateCache", "true")) && strategy != null;
|
||||
final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false"));
|
||||
|
||||
final Query<T> rewrittenQuery;
|
||||
if (populateCache) {
|
||||
rewrittenQuery = query.withOverriddenContext(ImmutableMap.of("bySegment", "true", "intermediate", "true"));
|
||||
} else {
|
||||
rewrittenQuery = query.withOverriddenContext(ImmutableMap.of("intermediate", "true"));
|
||||
}
|
||||
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline = serverView.getTimeline(query.getDataSource());
|
||||
if (timeline == null) {
|
||||
return Sequences.empty();
|
||||
}
|
||||
|
||||
byte[] queryCacheKey = null;
|
||||
if (strategy != null) {
|
||||
queryCacheKey = strategy.computeCacheKey(query);
|
||||
}
|
||||
|
||||
for (Interval interval : rewrittenQuery.getIntervals()) {
|
||||
List<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline.lookup(interval);
|
||||
|
||||
for (TimelineObjectHolder<String, ServerSelector> holder : serversLookup) {
|
||||
for (PartitionChunk<ServerSelector> chunk : holder.getObject()) {
|
||||
ServerSelector selector = chunk.getObject();
|
||||
final SegmentDescriptor descriptor = new SegmentDescriptor(
|
||||
holder.getInterval(), holder.getVersion(), chunk.getChunkNumber()
|
||||
);
|
||||
|
||||
if (queryCacheKey == null) {
|
||||
final DruidServer server = selector.pick();
|
||||
List<SegmentDescriptor> descriptors = segs.get(server);
|
||||
|
||||
if (descriptors == null) {
|
||||
descriptors = Lists.newArrayList();
|
||||
segs.put(server, descriptors);
|
||||
}
|
||||
|
||||
descriptors.add(descriptor);
|
||||
}
|
||||
else {
|
||||
final Interval segmentQueryInterval = holder.getInterval();
|
||||
final byte[] versionBytes = descriptor.getVersion().getBytes();
|
||||
|
||||
final byte[] cacheKey = ByteBuffer
|
||||
.allocate(16 + versionBytes.length + 4 + queryCacheKey.length)
|
||||
.putLong(segmentQueryInterval.getStartMillis())
|
||||
.putLong(segmentQueryInterval.getEndMillis())
|
||||
.put(versionBytes)
|
||||
.putInt(descriptor.getPartitionNumber())
|
||||
.put(queryCacheKey)
|
||||
.array();
|
||||
final String segmentIdentifier = selector.getSegment().getIdentifier();
|
||||
final Cache cache = cacheBroker.provideCache(segmentIdentifier);
|
||||
final byte[] cachedValue = cache.get(cacheKey);
|
||||
|
||||
if (useCache && cachedValue != null) {
|
||||
cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue));
|
||||
} else {
|
||||
final DruidServer server = selector.pick();
|
||||
List<SegmentDescriptor> descriptors = segs.get(server);
|
||||
|
||||
if (descriptors == null) {
|
||||
descriptors = Lists.newArrayList();
|
||||
segs.put(server, descriptors);
|
||||
}
|
||||
|
||||
descriptors.add(descriptor);
|
||||
cachePopulatorMap.put(
|
||||
String.format("%s_%s", segmentIdentifier, segmentQueryInterval),
|
||||
new CachePopulator(cache, objectMapper, cacheKey)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new LazySequence<T>(
|
||||
new Supplier<Sequence<T>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<T> get()
|
||||
{
|
||||
ArrayList<Pair<DateTime, Sequence<T>>> listOfSequences = Lists.newArrayList();
|
||||
|
||||
addSequencesFromServer(listOfSequences);
|
||||
addSequencesFromCache(listOfSequences);
|
||||
|
||||
Collections.sort(
|
||||
listOfSequences,
|
||||
Ordering.natural().onResultOf(Pair.<DateTime, Sequence<T>>lhsFn())
|
||||
);
|
||||
|
||||
final Sequence<Sequence<T>> seq = Sequences.simple(
|
||||
Iterables.transform(listOfSequences, Pair.<DateTime, Sequence<T>>rhsFn())
|
||||
);
|
||||
if (strategy == null) {
|
||||
return toolChest.mergeSequences(seq);
|
||||
}
|
||||
else {
|
||||
return strategy.mergeSequences(seq);
|
||||
}
|
||||
}
|
||||
|
||||
private void addSequencesFromCache(ArrayList<Pair<DateTime, Sequence<T>>> listOfSequences)
|
||||
{
|
||||
if (strategy == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Function<Object, T> pullFromCacheFunction = strategy.pullFromCache();
|
||||
for (Pair<DateTime, byte[]> cachedResultPair : cachedResults) {
|
||||
final byte[] cachedResult = cachedResultPair.rhs;
|
||||
Sequence<Object> cachedSequence = new BaseSequence<Object, Iterator<Object>>(
|
||||
new BaseSequence.IteratorMaker<Object, Iterator<Object>>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<Object> make()
|
||||
{
|
||||
try {
|
||||
return objectMapper.readValues(
|
||||
objectMapper.getJsonFactory().createJsonParser(cachedResult), Object.class
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup(Iterator<Object> iterFromMake)
|
||||
{
|
||||
}
|
||||
}
|
||||
);
|
||||
listOfSequences.add(Pair.of(cachedResultPair.lhs, Sequences.map(cachedSequence, pullFromCacheFunction)));
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void addSequencesFromServer(ArrayList<Pair<DateTime, Sequence<T>>> listOfSequences)
|
||||
{
|
||||
for (Map.Entry<DruidServer, List<SegmentDescriptor>> entry : segs.entrySet()) {
|
||||
final DruidServer server = entry.getKey();
|
||||
final List<SegmentDescriptor> descriptors = entry.getValue();
|
||||
|
||||
final QueryRunner clientQueryable = serverView.getQueryRunner(server);
|
||||
if (clientQueryable == null) {
|
||||
throw new ISE("WTF!? server[%s] doesn't have a client Queryable?", server);
|
||||
}
|
||||
|
||||
final Sequence<T> resultSeqToAdd;
|
||||
final MultipleSpecificSegmentSpec segmentSpec = new MultipleSpecificSegmentSpec(descriptors);
|
||||
List<Interval> intervals = segmentSpec.getIntervals();
|
||||
|
||||
if ("realtime".equals(server.getType()) || !populateCache || isBySegment) {
|
||||
resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec));
|
||||
} else {
|
||||
resultSeqToAdd = toolChest.mergeSequences(
|
||||
Sequences.map(
|
||||
clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec)),
|
||||
new Function<Object, Sequence<T>>()
|
||||
{
|
||||
private final Function<T, Object> prepareForCache = strategy.prepareForCache();
|
||||
|
||||
@Override
|
||||
public Sequence<T> apply(Object input)
|
||||
{
|
||||
Result<Object> result = (Result<Object>) input;
|
||||
final BySegmentResultValueClass<T> value = (BySegmentResultValueClass<T>) result.getValue();
|
||||
String segmentIdentifier = value.getSegmentId();
|
||||
final Iterable<T> segmentResults = value.getResults();
|
||||
|
||||
cachePopulatorMap.get(
|
||||
String.format("%s_%s", segmentIdentifier, value.getIntervalString())
|
||||
).populate(Iterables.transform(segmentResults, prepareForCache));
|
||||
|
||||
return Sequences.simple(
|
||||
Iterables.transform(
|
||||
segmentResults,
|
||||
toolChest.makeMetricManipulatorFn(
|
||||
rewrittenQuery,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.deserialize(object);
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
listOfSequences.add(Pair.of(intervals.get(0).getStart(), resultSeqToAdd));
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private static class CachePopulator
|
||||
{
|
||||
private final Cache cache;
|
||||
private final ObjectMapper mapper;
|
||||
private final byte[] key;
|
||||
|
||||
public CachePopulator(Cache cache, ObjectMapper mapper, byte[] key)
|
||||
{
|
||||
this.cache = cache;
|
||||
this.mapper = mapper;
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
public void populate(Iterable<Object> results)
|
||||
{
|
||||
try {
|
||||
List<byte[]> bytes = Lists.newArrayList();
|
||||
int size = 0;
|
||||
for (Object result : results) {
|
||||
final byte[] array = mapper.writeValueAsBytes(result);
|
||||
size += array.length;
|
||||
bytes.add(array);
|
||||
}
|
||||
|
||||
byte[] valueBytes = new byte[size];
|
||||
int offset = 0;
|
||||
for (byte[] array : bytes) {
|
||||
System.arraycopy(array, 0, valueBytes, offset, array.length);
|
||||
offset += array.length;
|
||||
}
|
||||
|
||||
cache.put(key, valueBytes);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,22 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ClientConfig
|
||||
{
|
||||
@Config("druid.zk.paths.announcementsPath")
|
||||
public abstract String getAnnouncementsPath();
|
||||
|
||||
@Config("druid.zk.paths.servedSegmentsPath")
|
||||
public abstract String getServedSegmentsPath();
|
||||
|
||||
public InventoryManagerConfig getClientInventoryManagerConfig()
|
||||
{
|
||||
return new InventoryManagerConfig(
|
||||
getAnnouncementsPath(),
|
||||
getServedSegmentsPath()
|
||||
);
|
||||
}
|
||||
}
|
@ -0,0 +1,144 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientInventoryManager extends InventoryManager<DruidServer>
|
||||
{
|
||||
private static final Logger log = new Logger(ClientInventoryManager.class);
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final Executor exec;
|
||||
|
||||
private final MutableServerView serverView;
|
||||
|
||||
public ClientInventoryManager(
|
||||
final InventoryManagerConfig config,
|
||||
final PhoneBook yp,
|
||||
final MutableServerView serverView
|
||||
)
|
||||
{
|
||||
super(log, config, yp);
|
||||
|
||||
this.serverView = serverView;
|
||||
|
||||
this.exec = Executors.newFixedThreadPool(
|
||||
1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("CIV-Execution-%d").build()
|
||||
);
|
||||
|
||||
setStrategy(
|
||||
new InventoryManagementStrategy<DruidServer>()
|
||||
{
|
||||
@Override
|
||||
public Class<DruidServer> getContainerClass()
|
||||
{
|
||||
return DruidServer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<String, PhoneBookPeon<?>> makeSubListener(final DruidServer server)
|
||||
{
|
||||
ClientInventoryManager.this.serverView.addServer(server);
|
||||
|
||||
return Pair.<String, PhoneBookPeon<?>>of(
|
||||
server.getName(),
|
||||
new PhoneBookPeon<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public Class<DataSegment> getObjectClazz()
|
||||
{
|
||||
return DataSegment.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newEntry(String segmentId, DataSegment segment)
|
||||
{
|
||||
exec.execute(new AddSegmentRunnable(server, segment));
|
||||
server.addDataSegment(segmentId, segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void entryRemoved(String segmentId)
|
||||
{
|
||||
exec.execute(new RemoveSegmentRunnable(server, segmentId));
|
||||
server.removeDataSegment(segmentId);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void objectRemoved(DruidServer server)
|
||||
{
|
||||
ClientInventoryManager.this.serverView.removeServer(server);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doesSerde()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidServer deserialize(String name, Map<String, String> properties)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doStop()
|
||||
{
|
||||
synchronized (lock) {
|
||||
serverView.clear();
|
||||
}
|
||||
}
|
||||
|
||||
private class RemoveSegmentRunnable implements Runnable
|
||||
{
|
||||
private final DruidServer server;
|
||||
private final String segmentId;
|
||||
|
||||
public RemoveSegmentRunnable(DruidServer server, String segmentId)
|
||||
{
|
||||
this.server = server;
|
||||
this.segmentId = segmentId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
serverView.serverRemovedSegment(server, segmentId);
|
||||
}
|
||||
}
|
||||
|
||||
private class AddSegmentRunnable implements Runnable
|
||||
{
|
||||
private final DruidServer server;
|
||||
private final DataSegment segment;
|
||||
|
||||
public AddSegmentRunnable(DruidServer server, DataSegment segment)
|
||||
{
|
||||
this.server = server;
|
||||
this.segment = segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
serverView.serverAddedSegment(server, segment);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,261 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.client.selector.ServerSelector;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientSideServerView implements MutableServerView
|
||||
{
|
||||
private static final Logger log = new Logger(ClientSideServerView.class);
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final ConcurrentMap<DruidServer, DirectDruidClient> clients;
|
||||
private final Map<String, ServerSelector> selectors;
|
||||
private final Map<String, VersionedIntervalTimeline<String, ServerSelector>> timelines;
|
||||
private final ConcurrentMap<ServerCallback, Executor> serverCallbacks;
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks;
|
||||
private final QueryToolChestWarehouse warehose;
|
||||
private final ObjectMapper smileMapper;
|
||||
private final HttpClient httpClient;
|
||||
|
||||
public ClientSideServerView(
|
||||
QueryToolChestWarehouse warehose,
|
||||
ObjectMapper smileMapper,
|
||||
HttpClient httpClient
|
||||
)
|
||||
{
|
||||
this.warehose = warehose;
|
||||
this.smileMapper = smileMapper;
|
||||
this.httpClient = httpClient;
|
||||
|
||||
this.clients = Maps.newConcurrentMap();
|
||||
this.selectors = Maps.newHashMap();
|
||||
this.timelines = Maps.newHashMap();
|
||||
this.serverCallbacks = Maps.newConcurrentMap();
|
||||
this.segmentCallbacks = Maps.newConcurrentMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear()
|
||||
{
|
||||
synchronized (lock) {
|
||||
final Iterator<DruidServer> clientsIter = clients.keySet().iterator();
|
||||
while (clientsIter.hasNext()) {
|
||||
DruidServer server = clientsIter.next();
|
||||
clientsIter.remove();
|
||||
runServerCallbacks(server);
|
||||
}
|
||||
|
||||
timelines.clear();
|
||||
|
||||
final Iterator<ServerSelector> selectorsIter = selectors.values().iterator();
|
||||
while (selectorsIter.hasNext()) {
|
||||
final ServerSelector selector = selectorsIter.next();
|
||||
selectorsIter.remove();
|
||||
while (!selector.isEmpty()) {
|
||||
final DruidServer pick = selector.pick();
|
||||
runSegmentCallbacks(
|
||||
new Function<SegmentCallback, CallbackAction>()
|
||||
{
|
||||
@Override
|
||||
public CallbackAction apply(@Nullable SegmentCallback input)
|
||||
{
|
||||
return input.segmentRemoved(pick, selector.getSegment());
|
||||
}
|
||||
}
|
||||
);
|
||||
selector.removeServer(pick);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addServer(DruidServer server)
|
||||
{
|
||||
QueryRunner exists = clients.put(server, new DirectDruidClient(warehose, smileMapper, httpClient, server.getHost()));
|
||||
if (exists != null) {
|
||||
log.warn("QueryRunner for server[%s] already existed!?", server);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeServer(DruidServer server)
|
||||
{
|
||||
clients.remove(server);
|
||||
for (DataSegment segment : server.getSegments().values()) {
|
||||
serverRemovedSegment(server, segment.getIdentifier());
|
||||
}
|
||||
runServerCallbacks(server);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serverAddedSegment(final DruidServer server, final DataSegment segment)
|
||||
{
|
||||
String segmentId = segment.getIdentifier();
|
||||
synchronized (lock) {
|
||||
log.info("Adding segment[%s] for server[%s]", segment, server);
|
||||
|
||||
ServerSelector selector = selectors.get(segmentId);
|
||||
if (selector == null) {
|
||||
selector = new ServerSelector(segment);
|
||||
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource());
|
||||
if (timeline == null) {
|
||||
timeline = new VersionedIntervalTimeline<String, ServerSelector>(Ordering.natural());
|
||||
timelines.put(segment.getDataSource(), timeline);
|
||||
}
|
||||
|
||||
timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector));
|
||||
selectors.put(segmentId, selector);
|
||||
}
|
||||
|
||||
selector.addServer(server);
|
||||
|
||||
runSegmentCallbacks(
|
||||
new Function<SegmentCallback, CallbackAction>()
|
||||
{
|
||||
@Override
|
||||
public CallbackAction apply(@Nullable SegmentCallback input)
|
||||
{
|
||||
return input.segmentAdded(server, segment);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serverRemovedSegment(final DruidServer server, final String segmentId)
|
||||
{
|
||||
final ServerSelector selector;
|
||||
|
||||
synchronized (lock) {
|
||||
log.info("Removing segment[%s] from server[%s].", segmentId, server);
|
||||
|
||||
selector = selectors.get(segmentId);
|
||||
if (selector == null) {
|
||||
log.warn("Told to remove non-existant segment[%s]", segmentId);
|
||||
return;
|
||||
}
|
||||
|
||||
if (!selector.removeServer(server)) {
|
||||
log.warn(
|
||||
"Asked to disassociate non-existant association between server[%s] and segment[%s]",
|
||||
server,
|
||||
segmentId
|
||||
);
|
||||
}
|
||||
|
||||
if (selector.isEmpty()) {
|
||||
DataSegment segment = selector.getSegment();
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource());
|
||||
selectors.remove(segmentId);
|
||||
|
||||
if (timeline.remove(
|
||||
segment.getInterval(),
|
||||
segment.getVersion(),
|
||||
segment.getShardSpec().createChunk(selector)
|
||||
) == null) {
|
||||
log.warn(
|
||||
"Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist",
|
||||
segment.getInterval(),
|
||||
segment.getVersion()
|
||||
);
|
||||
}
|
||||
runSegmentCallbacks(
|
||||
new Function<SegmentCallback, CallbackAction>()
|
||||
{
|
||||
@Override
|
||||
public CallbackAction apply(@Nullable SegmentCallback input)
|
||||
{
|
||||
return input.segmentRemoved(server, selector.getSegment());
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource)
|
||||
{
|
||||
synchronized (lock) {
|
||||
return timelines.get(dataSource);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunner(DruidServer server)
|
||||
{
|
||||
synchronized (lock) {
|
||||
return clients.get(server);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerServerCallback(Executor exec, ServerCallback callback)
|
||||
{
|
||||
serverCallbacks.put(callback, exec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
|
||||
{
|
||||
segmentCallbacks.put(callback, exec);
|
||||
}
|
||||
|
||||
private void runSegmentCallbacks(
|
||||
final Function<SegmentCallback, CallbackAction> fn
|
||||
)
|
||||
{
|
||||
for (final Map.Entry<SegmentCallback, Executor> entry : segmentCallbacks.entrySet()) {
|
||||
entry.getValue().execute(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) {
|
||||
segmentCallbacks.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void runServerCallbacks(final DruidServer server)
|
||||
{
|
||||
for (final Map.Entry<ServerCallback, Executor> entry : serverCallbacks.entrySet()) {
|
||||
entry.getValue().execute(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
if (CallbackAction.UNREGISTER == entry.getKey().serverRemoved(server)) {
|
||||
serverCallbacks.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
357
client/src/main/java/com/metamx/druid/client/DataSegment.java
Normal file
357
client/src/main/java/com/metamx/druid/client/DataSegment.java
Normal file
@ -0,0 +1,357 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Granularity;
|
||||
import com.metamx.druid.jackson.CommaListJoinDeserializer;
|
||||
import com.metamx.druid.jackson.CommaListJoinSerializer;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.druid.shard.ShardSpec;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.map.annotate.JsonDeserialize;
|
||||
import org.codehaus.jackson.map.annotate.JsonSerialize;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DataSegment implements Comparable<DataSegment>
|
||||
{
|
||||
public static String delimiter = "_";
|
||||
public static String makeDataSegmentIdentifier(
|
||||
String dataSource,
|
||||
DateTime start,
|
||||
DateTime end,
|
||||
String version,
|
||||
ShardSpec shardSpec
|
||||
)
|
||||
{
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
sb.append(dataSource).append(delimiter)
|
||||
.append(start).append(delimiter)
|
||||
.append(end).append(delimiter)
|
||||
.append(version);
|
||||
|
||||
if (shardSpec.getPartitionNum() != 0) {
|
||||
sb.append(delimiter).append(shardSpec.getPartitionNum());
|
||||
}
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
private final String version;
|
||||
private final Map<String, Object> loadSpec;
|
||||
private final List<String> dimensions;
|
||||
private final List<String> metrics;
|
||||
private final ShardSpec shardSpec;
|
||||
private final long size;
|
||||
private final String identifier;
|
||||
|
||||
@JsonCreator
|
||||
public DataSegment(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval,
|
||||
@JsonProperty("version") String version,
|
||||
@JsonProperty("loadSpec") Map<String, Object> loadSpec,
|
||||
@JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> dimensions,
|
||||
@JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> metrics,
|
||||
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
||||
@JsonProperty("size") long size
|
||||
)
|
||||
{
|
||||
final Predicate<String> nonEmpty = new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable String input)
|
||||
{
|
||||
return input != null && !input.isEmpty();
|
||||
}
|
||||
};
|
||||
|
||||
this.dataSource = dataSource.toLowerCase();
|
||||
this.interval = interval;
|
||||
this.loadSpec = loadSpec;
|
||||
this.version = version;
|
||||
this.dimensions = dimensions == null
|
||||
? ImmutableList.<String>of()
|
||||
: ImmutableList.copyOf(Iterables.filter(dimensions, nonEmpty));
|
||||
this.metrics = metrics == null
|
||||
? ImmutableList.<String>of()
|
||||
: ImmutableList.copyOf(Iterables.filter(metrics, nonEmpty));
|
||||
this.shardSpec = (shardSpec == null) ? new NoneShardSpec() : shardSpec;
|
||||
this.size = size;
|
||||
|
||||
this.identifier = makeDataSegmentIdentifier(
|
||||
this.dataSource,
|
||||
this.interval.getStart(),
|
||||
this.interval.getEnd(),
|
||||
this.version,
|
||||
this.shardSpec
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get dataSource
|
||||
*
|
||||
* @return the dataSource
|
||||
*/
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Object> getLoadSpec()
|
||||
{
|
||||
return loadSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonSerialize(using = CommaListJoinSerializer.class)
|
||||
public List<String> getDimensions()
|
||||
{
|
||||
return dimensions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonSerialize(using = CommaListJoinSerializer.class)
|
||||
public List<String> getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ShardSpec getShardSpec()
|
||||
{
|
||||
return shardSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getSize()
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getIdentifier()
|
||||
{
|
||||
return identifier;
|
||||
}
|
||||
|
||||
public SegmentDescriptor toDescriptor()
|
||||
{
|
||||
return new SegmentDescriptor(interval, version, shardSpec.getPartitionNum());
|
||||
}
|
||||
|
||||
public DataSegment withLoadSpec(Map<String, Object> loadSpec)
|
||||
{
|
||||
return builder(this).loadSpec(loadSpec).build();
|
||||
}
|
||||
|
||||
public DataSegment withSize(long size)
|
||||
{
|
||||
return builder(this).size(size).build();
|
||||
}
|
||||
|
||||
public DataSegment withVersion(String version)
|
||||
{
|
||||
return builder(this).version(version).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(DataSegment dataSegment)
|
||||
{
|
||||
return getIdentifier().compareTo(dataSegment.getIdentifier());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (o instanceof DataSegment) {
|
||||
return getIdentifier().equals(((DataSegment) o).getIdentifier());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return getIdentifier().hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DataSegment{" +
|
||||
"size=" + size +
|
||||
", shardSpec=" + shardSpec +
|
||||
", metrics=" + metrics +
|
||||
", dimensions=" + dimensions +
|
||||
", version='" + version + '\'' +
|
||||
", loadSpec=" + loadSpec +
|
||||
", interval=" + interval +
|
||||
", dataSource='" + dataSource + '\'' +
|
||||
'}';
|
||||
}
|
||||
|
||||
public static Comparator<DataSegment> bucketMonthComparator()
|
||||
{
|
||||
return new Comparator<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public int compare(DataSegment lhs, DataSegment rhs)
|
||||
{
|
||||
int retVal;
|
||||
|
||||
DateTime lhsMonth = Granularity.MONTH.truncate(lhs.getInterval().getStart());
|
||||
DateTime rhsMonth = Granularity.MONTH.truncate(rhs.getInterval().getStart());
|
||||
|
||||
retVal = lhsMonth.compareTo(rhsMonth);
|
||||
|
||||
if (retVal != 0) {
|
||||
return retVal;
|
||||
}
|
||||
|
||||
return lhs.compareTo(rhs);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static Builder builder(DataSegment segment)
|
||||
{
|
||||
return new Builder(segment);
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private String dataSource;
|
||||
private Interval interval;
|
||||
private String version;
|
||||
private Map<String, Object> loadSpec;
|
||||
private List<String> dimensions;
|
||||
private List<String> metrics;
|
||||
private ShardSpec shardSpec;
|
||||
private long size;
|
||||
|
||||
public Builder()
|
||||
{
|
||||
this.loadSpec = ImmutableMap.of();
|
||||
this.dimensions = ImmutableList.of();
|
||||
this.metrics = ImmutableList.of();
|
||||
this.shardSpec = new NoneShardSpec();
|
||||
this.size = -1;
|
||||
}
|
||||
|
||||
public Builder(DataSegment segment)
|
||||
{
|
||||
this.dataSource = segment.getDataSource();
|
||||
this.interval = segment.getInterval();
|
||||
this.version = segment.getVersion();
|
||||
this.loadSpec = segment.getLoadSpec();
|
||||
this.dimensions = segment.getDimensions();
|
||||
this.metrics = segment.getMetrics();
|
||||
this.shardSpec = segment.getShardSpec();
|
||||
this.size = segment.getSize();
|
||||
}
|
||||
|
||||
public Builder dataSource(String dataSource)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder interval(Interval interval)
|
||||
{
|
||||
this.interval = interval;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder version(String version)
|
||||
{
|
||||
this.version = version;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder loadSpec(Map<String, Object> loadSpec)
|
||||
{
|
||||
this.loadSpec = loadSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder dimensions(List<String> dimensions)
|
||||
{
|
||||
this.dimensions = dimensions;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder metrics(List<String> metrics)
|
||||
{
|
||||
this.metrics = metrics;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder shardSpec(ShardSpec shardSpec)
|
||||
{
|
||||
this.shardSpec = shardSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder size(long size)
|
||||
{
|
||||
this.size = size;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DataSegment build()
|
||||
{
|
||||
// Check stuff that goes into the identifier, at least.
|
||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
Preconditions.checkNotNull(interval, "interval");
|
||||
Preconditions.checkNotNull(version, "version");
|
||||
Preconditions.checkNotNull(shardSpec, "shardSpec");
|
||||
|
||||
return new DataSegment(
|
||||
dataSource,
|
||||
interval,
|
||||
version,
|
||||
loadSpec,
|
||||
dimensions,
|
||||
metrics,
|
||||
shardSpec,
|
||||
size
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,268 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.query.MetricManipulationFn;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChest;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||
import com.metamx.druid.result.Result;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import com.metamx.http.client.io.AppendableByteArrayInputStream;
|
||||
import com.metamx.http.client.response.ClientResponse;
|
||||
import com.metamx.http.client.response.InputStreamResponseHandler;
|
||||
import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.codehaus.jackson.ObjectCodec;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.type.TypeFactory;
|
||||
import org.codehaus.jackson.smile.SmileFactory;
|
||||
import org.codehaus.jackson.type.JavaType;
|
||||
import org.jboss.netty.handler.codec.http.HttpChunk;
|
||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DirectDruidClient<T> implements QueryRunner<T>
|
||||
{
|
||||
private static final Logger log = new Logger(DirectDruidClient.class);
|
||||
|
||||
private static final Map<Class<? extends Query>, Pair<JavaType, JavaType>> typesMap = Maps.newConcurrentMap();
|
||||
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final HttpClient httpClient;
|
||||
private final String host;
|
||||
|
||||
private final boolean isSmile;
|
||||
|
||||
public DirectDruidClient(
|
||||
QueryToolChestWarehouse warehouse,
|
||||
ObjectMapper objectMapper,
|
||||
HttpClient httpClient,
|
||||
String host
|
||||
)
|
||||
{
|
||||
this.warehouse = warehouse;
|
||||
this.objectMapper = objectMapper;
|
||||
this.httpClient = httpClient;
|
||||
this.host = host;
|
||||
|
||||
isSmile = this.objectMapper.getJsonFactory() instanceof SmileFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(Query<T> query)
|
||||
{
|
||||
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
||||
boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false"));
|
||||
|
||||
Pair<JavaType, JavaType> types = typesMap.get(query.getClass());
|
||||
if (types == null) {
|
||||
final TypeFactory typeFactory = objectMapper.getTypeFactory();
|
||||
JavaType baseType = typeFactory.constructType(toolChest.getResultTypeReference());
|
||||
JavaType bySegmentType = typeFactory.constructParametricType(
|
||||
Result.class, typeFactory.constructParametricType(BySegmentResultValueClass.class, baseType)
|
||||
);
|
||||
types = Pair.of(baseType, bySegmentType);
|
||||
typesMap.put(query.getClass(), types);
|
||||
}
|
||||
|
||||
final JavaType typeRef;
|
||||
if (isBySegment) {
|
||||
typeRef = types.rhs;
|
||||
}
|
||||
else {
|
||||
typeRef = types.lhs;
|
||||
}
|
||||
|
||||
final Future<InputStream> future;
|
||||
final String url = String.format("http://%s/druid/v2/", host);
|
||||
|
||||
try {
|
||||
log.debug("Querying url[%s]", url);
|
||||
future = httpClient
|
||||
.post(new URL(url))
|
||||
.setContent(objectMapper.writeValueAsBytes(query))
|
||||
.setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? "application/smile" : "application/json")
|
||||
.go(
|
||||
new InputStreamResponseHandler()
|
||||
{
|
||||
|
||||
long startTime;
|
||||
long byteCount = 0;
|
||||
|
||||
@Override
|
||||
public ClientResponse<AppendableByteArrayInputStream> handleResponse(HttpResponse response)
|
||||
{
|
||||
log.debug("Initial response from url[%s]", url);
|
||||
startTime = System.currentTimeMillis();
|
||||
byteCount += response.getContent().readableBytes();
|
||||
return super.handleResponse(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientResponse<AppendableByteArrayInputStream> handleChunk(
|
||||
ClientResponse<AppendableByteArrayInputStream> clientResponse, HttpChunk chunk
|
||||
)
|
||||
{
|
||||
final int bytes = chunk.getContent().readableBytes();
|
||||
byteCount += bytes;
|
||||
return super.handleChunk(clientResponse, chunk);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientResponse<InputStream> done(ClientResponse<AppendableByteArrayInputStream> clientResponse)
|
||||
{
|
||||
long stopTime = System.currentTimeMillis();
|
||||
log.debug(
|
||||
"Completed request to url[%s] with %,d bytes returned in %,d millis [%,f b/s].",
|
||||
url,
|
||||
byteCount,
|
||||
stopTime - startTime,
|
||||
byteCount / (0.0001 * (stopTime - startTime))
|
||||
);
|
||||
return super.done(clientResponse);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
Sequence<T> retVal = new BaseSequence<T, JsonParserIterator<T>>(
|
||||
new BaseSequence.IteratorMaker<T, JsonParserIterator<T>>()
|
||||
{
|
||||
@Override
|
||||
public JsonParserIterator<T> make()
|
||||
{
|
||||
return new JsonParserIterator<T>(typeRef, future);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup(JsonParserIterator<T> iterFromMake)
|
||||
{
|
||||
Closeables.closeQuietly(iterFromMake);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
if (!isBySegment) {
|
||||
retVal = Sequences.map(
|
||||
retVal,
|
||||
toolChest.makeMetricManipulatorFn(query, new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.deserialize(object);
|
||||
}
|
||||
})
|
||||
);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private class JsonParserIterator<T> implements Iterator<T>, Closeable
|
||||
{
|
||||
private JsonParser jp;
|
||||
private ObjectCodec objectCodec;
|
||||
private final JavaType typeRef;
|
||||
private final Future<InputStream> future;
|
||||
|
||||
public JsonParserIterator(JavaType typeRef, Future<InputStream> future)
|
||||
{
|
||||
this.typeRef = typeRef;
|
||||
this.future = future;
|
||||
jp = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
init();
|
||||
|
||||
if (jp.isClosed()) {
|
||||
return false;
|
||||
}
|
||||
if (jp.getCurrentToken() == JsonToken.END_ARRAY) {
|
||||
Closeables.closeQuietly(jp);
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T next()
|
||||
{
|
||||
init();
|
||||
try {
|
||||
final T retVal = objectCodec.readValue(jp, typeRef);
|
||||
jp.nextToken();
|
||||
return retVal;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
private void init()
|
||||
{
|
||||
if (jp == null) {
|
||||
try {
|
||||
jp = objectMapper.getJsonFactory().createJsonParser(future.get());
|
||||
if (jp.nextToken() != JsonToken.START_ARRAY) {
|
||||
throw new IAE("Next token wasn't a START_ARRAY, was[%s]", jp.getCurrentToken());
|
||||
} else {
|
||||
jp.nextToken();
|
||||
objectCodec = jp.getCodec();
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
jp.close();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,100 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DruidDataSource
|
||||
{
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final String name;
|
||||
private final Map<String, String> properties;
|
||||
private final Map<String, DataSegment> partitionNames;
|
||||
private final ConcurrentSkipListSet<DataSegment> segmentsHolder;
|
||||
|
||||
public DruidDataSource(
|
||||
String name,
|
||||
Map<String, String> properties
|
||||
)
|
||||
{
|
||||
this.name = name;
|
||||
this.properties = properties;
|
||||
|
||||
this.partitionNames = Maps.newHashMap();
|
||||
this.segmentsHolder = new ConcurrentSkipListSet<DataSegment>();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, String> getProperties()
|
||||
{
|
||||
return properties;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
return Collections.unmodifiableSet(segmentsHolder);
|
||||
}
|
||||
|
||||
public DruidDataSource addSegment(String partitionName, DataSegment dataSegment)
|
||||
{
|
||||
synchronized (lock) {
|
||||
partitionNames.put(partitionName, dataSegment);
|
||||
|
||||
segmentsHolder.add(dataSegment);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public DruidDataSource addSegments(Map<String, DataSegment> partitionMap)
|
||||
{
|
||||
synchronized (lock) {
|
||||
partitionNames.putAll(partitionMap);
|
||||
|
||||
segmentsHolder.addAll(partitionMap.values());
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public DruidDataSource removePartition(String partitionName)
|
||||
{
|
||||
synchronized (lock) {
|
||||
DataSegment dataPart = partitionNames.remove(partitionName);
|
||||
|
||||
if (dataPart == null) {
|
||||
return this;
|
||||
}
|
||||
|
||||
segmentsHolder.remove(dataPart);
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean isEmpty()
|
||||
{
|
||||
return segmentsHolder.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DruidDataSource{" +
|
||||
"properties=" + properties +
|
||||
", partitions=" + segmentsHolder.toString() +
|
||||
'}';
|
||||
}
|
||||
}
|
225
client/src/main/java/com/metamx/druid/client/DruidServer.java
Normal file
225
client/src/main/java/com/metamx/druid/client/DruidServer.java
Normal file
@ -0,0 +1,225 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DruidServer implements Comparable
|
||||
{
|
||||
private static final Logger log = new Logger(DruidServer.class);
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final String name;
|
||||
private final ConcurrentMap<String, DruidDataSource> dataSources;
|
||||
private final Map<String, DataSegment> segments;
|
||||
|
||||
private final String host;
|
||||
private final long maxSize;
|
||||
private final String type;
|
||||
|
||||
private volatile long currSize;
|
||||
|
||||
public DruidServer(
|
||||
DruidServerConfig config
|
||||
)
|
||||
{
|
||||
this(
|
||||
config.getServerName(),
|
||||
config.getHost(),
|
||||
config.getMaxSize(),
|
||||
config.getType()
|
||||
);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public DruidServer(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("host") String host,
|
||||
@JsonProperty("maxSize") long maxSize,
|
||||
@JsonProperty("type") String type
|
||||
)
|
||||
{
|
||||
this.name = name;
|
||||
this.host = host;
|
||||
this.maxSize = maxSize;
|
||||
this.type = type;
|
||||
|
||||
this.dataSources = new ConcurrentHashMap<String, DruidDataSource>();
|
||||
this.segments = new HashMap<String, DataSegment>();
|
||||
}
|
||||
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
public Map<String, String> getStringProps()
|
||||
{
|
||||
return ImmutableMap.of(
|
||||
"name", name,
|
||||
"host", host,
|
||||
"maxSize", String.valueOf(maxSize),
|
||||
"type", type
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getHost()
|
||||
{
|
||||
return host;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getCurrSize()
|
||||
{
|
||||
return currSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxSize()
|
||||
{
|
||||
return maxSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
public DataSegment getSegment(String segmentName)
|
||||
{
|
||||
return segments.get(segmentName);
|
||||
}
|
||||
|
||||
public DruidServer addDataSegment(String segmentName, DataSegment segment)
|
||||
{
|
||||
synchronized (lock) {
|
||||
String dataSourceName = segment.getDataSource();
|
||||
DruidDataSource dataSource = dataSources.get(dataSourceName);
|
||||
|
||||
if (dataSource == null) {
|
||||
dataSource = new DruidDataSource(
|
||||
dataSourceName,
|
||||
ImmutableMap.of("client", "side")
|
||||
);
|
||||
dataSources.put(dataSourceName, dataSource);
|
||||
}
|
||||
|
||||
dataSource.addSegment(segmentName, segment);
|
||||
segments.put(segmentName, segment);
|
||||
|
||||
currSize += segment.getSize();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public DruidServer removeDataSegment(String segmentName)
|
||||
{
|
||||
synchronized (lock) {
|
||||
DataSegment segment = segments.get(segmentName);
|
||||
|
||||
if (segment == null) {
|
||||
log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", name, segmentName);
|
||||
return this;
|
||||
}
|
||||
|
||||
DruidDataSource dataSource = dataSources.get(segment.getDataSource());
|
||||
|
||||
if (dataSource == null) {
|
||||
log.warn(
|
||||
"Asked to remove data segment from dataSource[%s] that doesn't exist, but the segment[%s] exists!?!?!?! wtf? server[%s]",
|
||||
segment.getDataSource(),
|
||||
segmentName,
|
||||
name
|
||||
);
|
||||
return this;
|
||||
}
|
||||
|
||||
dataSource.removePartition(segmentName);
|
||||
segments.remove(segmentName);
|
||||
if (dataSource.isEmpty()) {
|
||||
dataSources.remove(dataSource.getName());
|
||||
}
|
||||
currSize -= segment.getSize();
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public DruidDataSource getDataSource(String dataSource)
|
||||
{
|
||||
return dataSources.get(dataSource);
|
||||
}
|
||||
|
||||
public Iterable<DruidDataSource> getDataSources()
|
||||
{
|
||||
return dataSources.values();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DruidServer that = (DruidServer) o;
|
||||
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return name != null ? name.hashCode() : 0;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DruidServer{" +
|
||||
"name='" + name + '\'' +
|
||||
", host='" + host + '\'' +
|
||||
", maxSize=" + maxSize +
|
||||
", type=" + type +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return 0;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
return name.compareTo(((DruidServer) o).name);
|
||||
}
|
||||
}
|
@ -0,0 +1,22 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class DruidServerConfig
|
||||
{
|
||||
@Config("druid.host")
|
||||
public abstract String getServerName();
|
||||
|
||||
@Config("druid.host")
|
||||
public abstract String getHost();
|
||||
|
||||
@Config("druid.server.maxSize")
|
||||
public abstract long getMaxSize();
|
||||
|
||||
@Config("druid.server.type")
|
||||
@Default("historical")
|
||||
public abstract String getType();
|
||||
}
|
@ -0,0 +1,19 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface InventoryManagementStrategy<T>
|
||||
{
|
||||
public Class<T> getContainerClass();
|
||||
public Pair<String, PhoneBookPeon<?>> makeSubListener(final T baseObject);
|
||||
public void objectRemoved(final T baseObject);
|
||||
|
||||
// These are a hack to get around a poor serialization choice, please do not use
|
||||
public boolean doesSerde();
|
||||
public T deserialize(String name, Map<String, String> properties);
|
||||
}
|
@ -0,0 +1,201 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class InventoryManager<T>
|
||||
{
|
||||
private final Object lock = new Object();
|
||||
private volatile boolean started = false;
|
||||
|
||||
private final MasterPeon masterPeon;
|
||||
private final ConcurrentHashMap<String, PhoneBookPeon<?>> dataSourcePeons;
|
||||
private final ConcurrentHashMap<String, T> dataSources;
|
||||
|
||||
private final Logger log;
|
||||
private final PhoneBook yp;
|
||||
private final InventoryManagerConfig config;
|
||||
|
||||
private volatile InventoryManagementStrategy<T> strategy = null;
|
||||
|
||||
public InventoryManager(
|
||||
Logger log,
|
||||
InventoryManagerConfig config,
|
||||
PhoneBook zkPhoneBook
|
||||
)
|
||||
{
|
||||
this.log = log;
|
||||
this.config = config;
|
||||
this.yp = zkPhoneBook;
|
||||
|
||||
this.masterPeon = new MasterPeon();
|
||||
this.dataSourcePeons = new ConcurrentHashMap<String, PhoneBookPeon<?>>();
|
||||
this.dataSources = new ConcurrentHashMap<String, T>();
|
||||
}
|
||||
|
||||
public InventoryManager(
|
||||
Logger log,
|
||||
InventoryManagerConfig config,
|
||||
PhoneBook zkPhoneBook,
|
||||
InventoryManagementStrategy<T> strategy
|
||||
)
|
||||
{
|
||||
this(log, config, zkPhoneBook);
|
||||
setStrategy(strategy);
|
||||
}
|
||||
|
||||
public void setStrategy(InventoryManagementStrategy<T> strategy)
|
||||
{
|
||||
if (this.strategy != null) {
|
||||
throw new ISE("Management can only handle a single strategy, you cannot change your strategy.");
|
||||
}
|
||||
this.strategy = strategy;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
public void start()
|
||||
{
|
||||
synchronized (lock) {
|
||||
if (started) {
|
||||
return;
|
||||
}
|
||||
if (strategy == null) {
|
||||
throw new ISE("Management requires a strategy, please provide a strategy.");
|
||||
}
|
||||
|
||||
if (!yp.isStarted()) {
|
||||
throw new ISE("Management does not work without a running yellow pages.");
|
||||
}
|
||||
|
||||
yp.registerListener(config.getInventoryIdPath(), masterPeon);
|
||||
doStart();
|
||||
|
||||
started = true;
|
||||
}
|
||||
}
|
||||
|
||||
protected void doStart() {};
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
synchronized (lock) {
|
||||
if (!started) {
|
||||
return;
|
||||
}
|
||||
|
||||
yp.unregisterListener(config.getInventoryIdPath(), masterPeon);
|
||||
for (Map.Entry<String, PhoneBookPeon<?>> entry : dataSourcePeons.entrySet()) {
|
||||
yp.unregisterListener(
|
||||
yp.combineParts(Arrays.asList(config.getInventoryPath(), entry.getKey())), entry.getValue()
|
||||
);
|
||||
}
|
||||
|
||||
dataSources.clear();
|
||||
dataSourcePeons.clear();
|
||||
doStop();
|
||||
|
||||
started = false;
|
||||
}
|
||||
}
|
||||
|
||||
protected void doStop() {};
|
||||
|
||||
public boolean isStarted()
|
||||
{
|
||||
return started;
|
||||
}
|
||||
|
||||
public T getInventoryValue(String key)
|
||||
{
|
||||
return dataSources.get(key);
|
||||
}
|
||||
|
||||
public Collection<T> getInventory()
|
||||
{
|
||||
return dataSources.values();
|
||||
}
|
||||
|
||||
public void remove(List<String> nodePath)
|
||||
{
|
||||
yp.unpost(config.getInventoryIdPath(), yp.combineParts(nodePath));
|
||||
}
|
||||
|
||||
private class MasterPeon implements PhoneBookPeon
|
||||
{
|
||||
@Override
|
||||
public Class getObjectClazz()
|
||||
{
|
||||
return strategy.doesSerde() ? Object.class : strategy.getContainerClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newEntry(final String name, Object baseObject)
|
||||
{
|
||||
synchronized (lock) {
|
||||
if (!started) {
|
||||
return;
|
||||
}
|
||||
|
||||
log.info("New inventory container[%s]!", name);
|
||||
if (strategy.doesSerde()) { // Hack to work around poor serialization choice
|
||||
baseObject = strategy.deserialize(name, (Map<String, String>) baseObject);
|
||||
}
|
||||
Object shouldBeNull = dataSources.put(name, strategy.getContainerClass().cast(baseObject));
|
||||
if (shouldBeNull != null) {
|
||||
log.warn(
|
||||
"Just put key[%s] into dataSources and what was there wasn't null!? It was[%s]", name, shouldBeNull
|
||||
);
|
||||
}
|
||||
|
||||
Pair<String, PhoneBookPeon<?>> pair = strategy.makeSubListener(strategy.getContainerClass().cast(baseObject));
|
||||
|
||||
shouldBeNull = dataSourcePeons.put(pair.lhs, pair.rhs);
|
||||
if (shouldBeNull != null) {
|
||||
log.warn(
|
||||
"Just put key[%s] into dataSourcePeons and what was there wasn't null!? It was[%s]", name, shouldBeNull
|
||||
);
|
||||
}
|
||||
|
||||
String serviceName = yp.combineParts(Arrays.asList(config.getInventoryPath(), pair.lhs));
|
||||
log.info("Putting watch on [%s]", serviceName);
|
||||
yp.registerListener(serviceName, pair.rhs);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void entryRemoved(String name)
|
||||
{
|
||||
synchronized (lock) {
|
||||
if (!started) {
|
||||
return;
|
||||
}
|
||||
|
||||
log.info("Inventory container[%s] removed, deleting.", name);
|
||||
T removed = dataSources.remove(name);
|
||||
if (removed != null) {
|
||||
strategy.objectRemoved(removed);
|
||||
}
|
||||
else {
|
||||
log.warn("Removed empty element at path[%s]", name);
|
||||
}
|
||||
yp.unregisterListener(
|
||||
yp.combineParts(Arrays.asList(config.getInventoryPath(), name)), dataSourcePeons.remove(name)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class InventoryManagerConfig
|
||||
{
|
||||
private final String inventoryIdPath;
|
||||
private final String inventoryPath;
|
||||
|
||||
public InventoryManagerConfig(
|
||||
String inventoryIdPath,
|
||||
String inventoryPath
|
||||
)
|
||||
{
|
||||
this.inventoryIdPath = inventoryIdPath;
|
||||
this.inventoryPath = inventoryPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* The InventoryIdPath is the path where the InventoryManager should look for new inventory buckets. Because ZK does
|
||||
* not allow for children under ephemeral nodes, the common interaction for registering Inventory that might be
|
||||
* ephemeral is to
|
||||
*
|
||||
* 1) Create a permanent node underneath the InventoryPath
|
||||
* 2) Create an ephemeral node underneath the InventoryIdPath with the same name as the permanent nodes under InventoryPath
|
||||
* 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1)
|
||||
*
|
||||
* @return the inventoryIdPath
|
||||
*/
|
||||
public String getInventoryIdPath()
|
||||
{
|
||||
return inventoryIdPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* The InventoryPath is the path where the InventoryManager should look for new inventory.
|
||||
*
|
||||
* Because ZK does not allow for children under ephemeral nodes, the common interaction for registering an Inventory
|
||||
* container that might be ephemeral (like a server) is to
|
||||
*
|
||||
* 1) Create a permanent node underneath the InventoryPath
|
||||
* 2) Create an ephemeral node underneath the InventoryIdPath with the same name as the permanent nodes under InventoryPath
|
||||
* 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1)
|
||||
*
|
||||
* @return the inventoryIdPath
|
||||
*/
|
||||
public String getInventoryPath()
|
||||
{
|
||||
return inventoryPath;
|
||||
}
|
||||
}
|
@ -0,0 +1,16 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface MutableServerView extends ServerView
|
||||
{
|
||||
public void clear();
|
||||
|
||||
public void addServer(DruidServer server);
|
||||
|
||||
public void removeServer(DruidServer server);
|
||||
|
||||
public void serverAddedSegment(DruidServer server, DataSegment segment);
|
||||
|
||||
public void serverRemovedSegment(DruidServer server, String segmentId);
|
||||
}
|
@ -0,0 +1,98 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.client.selector.ServerSelector;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class OnlyNewSegmentWatcherServerView implements MutableServerView
|
||||
{
|
||||
private static final Logger log = new Logger(OnlyNewSegmentWatcherServerView.class);
|
||||
|
||||
private final ConcurrentMap<SegmentCallback, Executor> segmentCallbacks;
|
||||
|
||||
public OnlyNewSegmentWatcherServerView()
|
||||
{
|
||||
this.segmentCallbacks = Maps.newConcurrentMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addServer(DruidServer server)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeServer(DruidServer server)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serverAddedSegment(DruidServer server, DataSegment segment)
|
||||
{
|
||||
runSegmentCallbacks(server, segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serverRemovedSegment(DruidServer server, String segmentId)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner getQueryRunner(DruidServer server)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerServerCallback(Executor exec, ServerCallback callback)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
|
||||
{
|
||||
segmentCallbacks.put(callback, exec);
|
||||
}
|
||||
|
||||
private void runSegmentCallbacks(final DruidServer server, final DataSegment segment)
|
||||
{
|
||||
Iterator<Map.Entry<SegmentCallback, Executor>> iter = segmentCallbacks.entrySet().iterator();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
final Map.Entry<SegmentCallback, Executor> entry = iter.next();
|
||||
entry.getValue().execute(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
if (CallbackAction.UNREGISTER == entry.getKey().segmentAdded(server, segment)) {
|
||||
segmentCallbacks.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,87 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SegmentInventoryManager extends InventoryManager<DruidDataSource>
|
||||
{
|
||||
private static final Logger log = new Logger(SegmentInventoryManager.class);
|
||||
|
||||
public SegmentInventoryManager(
|
||||
SegmentInventoryManagerConfig config,
|
||||
PhoneBook zkPhoneBook
|
||||
)
|
||||
{
|
||||
super(
|
||||
log,
|
||||
new InventoryManagerConfig(
|
||||
config.getBasePath(),
|
||||
config.getBasePath()
|
||||
),
|
||||
zkPhoneBook,
|
||||
new SegmentInventoryManagementStrategy()
|
||||
);
|
||||
}
|
||||
|
||||
private static class SegmentInventoryManagementStrategy implements InventoryManagementStrategy<DruidDataSource>
|
||||
{
|
||||
@Override
|
||||
public Class<DruidDataSource> getContainerClass()
|
||||
{
|
||||
return DruidDataSource.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<String, PhoneBookPeon<?>> makeSubListener(final DruidDataSource baseObject)
|
||||
{
|
||||
return new Pair<String, PhoneBookPeon<?>>(
|
||||
baseObject.getName(),
|
||||
new PhoneBookPeon<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public Class<DataSegment> getObjectClazz()
|
||||
{
|
||||
return DataSegment.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newEntry(String name, DataSegment segment)
|
||||
{
|
||||
log.info("Adding dataSegment[%s].", segment);
|
||||
baseObject.addSegment(name, segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void entryRemoved(String name)
|
||||
{
|
||||
log.info("Partition[%s] deleted.", name);
|
||||
baseObject.removePartition(name);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void objectRemoved(DruidDataSource baseObject)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doesSerde()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidDataSource deserialize(String name, Map<String, String> properties)
|
||||
{
|
||||
return new DruidDataSource(name, properties);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,11 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class SegmentInventoryManagerConfig
|
||||
{
|
||||
@Config("druid.zk.paths.indexesPath")
|
||||
public abstract String getBasePath();
|
||||
}
|
@ -0,0 +1,125 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ServerInventoryManager extends InventoryManager<DruidServer>
|
||||
{
|
||||
private static final Map<String, Integer> removedSegments = Maps.newHashMap();
|
||||
|
||||
public ServerInventoryManager(
|
||||
ServerInventoryManagerConfig config,
|
||||
PhoneBook yp
|
||||
)
|
||||
{
|
||||
super(
|
||||
new Logger(ServerInventoryManager.class.getName() + "." + config.getServerInventoryPath()),
|
||||
new InventoryManagerConfig(
|
||||
config.getServerIdPath(),
|
||||
config.getServerInventoryPath()
|
||||
),
|
||||
yp,
|
||||
new ServerInventoryManagementStrategy(
|
||||
new Logger(
|
||||
ServerInventoryManager.class.getName() + "." + config.getServerInventoryPath()
|
||||
),
|
||||
config.getRemovedSegmentLifetime()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private static class ServerInventoryManagementStrategy implements InventoryManagementStrategy<DruidServer>
|
||||
{
|
||||
private final Logger log;
|
||||
private final int segmentLifetime;
|
||||
|
||||
ServerInventoryManagementStrategy(Logger log, int segmentLifetime)
|
||||
{
|
||||
this.log = log;
|
||||
this.segmentLifetime = segmentLifetime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<DruidServer> getContainerClass()
|
||||
{
|
||||
return DruidServer.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<String, PhoneBookPeon<?>> makeSubListener(final DruidServer druidServer)
|
||||
{
|
||||
return new Pair<String, PhoneBookPeon<?>>(
|
||||
druidServer.getName(),
|
||||
new PhoneBookPeon<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public Class<DataSegment> getObjectClazz()
|
||||
{
|
||||
return DataSegment.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newEntry(String name, DataSegment segment)
|
||||
{
|
||||
log.info("Server[%s] added new DataSegment[%s]", druidServer.getName(), segment);
|
||||
druidServer.addDataSegment(name, segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void entryRemoved(String name)
|
||||
{
|
||||
log.info("Entry [%s] deleted", name);
|
||||
removedSegments.put(druidServer.getSegment(name).getIdentifier(), segmentLifetime);
|
||||
druidServer.removeDataSegment(name);
|
||||
log.info("Server[%s] removed dataSegment[%s]", druidServer.getName(), name);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void objectRemoved(DruidServer baseObject)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doesSerde()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidServer deserialize(String name, Map<String, String> properties)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
public int lookupSegmentLifetime(DataSegment segment)
|
||||
{
|
||||
Integer lifetime = removedSegments.get(segment.getIdentifier());
|
||||
return (lifetime == null) ? 0 : lifetime;
|
||||
}
|
||||
|
||||
public void decrementRemovedSegmentsLifetime()
|
||||
{
|
||||
for (Iterator<Map.Entry<String, Integer>> mapIter = removedSegments.entrySet().iterator(); mapIter.hasNext(); ) {
|
||||
Map.Entry<String, Integer> segment = mapIter.next();
|
||||
int lifetime = segment.getValue() - 1;
|
||||
|
||||
if (lifetime < 0) {
|
||||
mapIter.remove();
|
||||
} else {
|
||||
segment.setValue(lifetime);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,19 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ServerInventoryManagerConfig
|
||||
{
|
||||
@Config("druid.zk.paths.announcementsPath")
|
||||
public abstract String getServerIdPath();
|
||||
|
||||
@Config("druid.zk.paths.servedSegmentsPath")
|
||||
public abstract String getServerInventoryPath();
|
||||
|
||||
@Config("druid.master.removedSegmentLifetime")
|
||||
@Default("1")
|
||||
public abstract int getRemovedSegmentLifetime();
|
||||
}
|
95
client/src/main/java/com/metamx/druid/client/ServerView.java
Normal file
95
client/src/main/java/com/metamx/druid/client/ServerView.java
Normal file
@ -0,0 +1,95 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.client.selector.ServerSelector;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ServerView
|
||||
{
|
||||
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(String dataSource);
|
||||
public <T> QueryRunner<T> getQueryRunner(DruidServer server);
|
||||
|
||||
public void registerServerCallback(Executor exec, ServerCallback callback);
|
||||
public void registerSegmentCallback(Executor exec, SegmentCallback callback);
|
||||
|
||||
public enum CallbackAction
|
||||
{
|
||||
CONTINUE,
|
||||
UNREGISTER,
|
||||
}
|
||||
|
||||
public static interface ServerCallback
|
||||
{
|
||||
/**
|
||||
* Called when a server is removed.
|
||||
*
|
||||
* The return value indicates if this callback has completed its work. Note that even if this callback
|
||||
* indicates that it should be unregistered, it is not possible to guarantee that this callback will not
|
||||
* get called again. There is a race condition between when this callback runs and other events that can cause
|
||||
* the callback to be queued for running. Thus, callbacks shouldn't assume that they will not get called
|
||||
* again after they are done. The contract is that the callback will eventually be unregistered, enforcing
|
||||
* a happens-before relationship is not part of the contract.
|
||||
*
|
||||
* @param server The server that was removed.
|
||||
* @return UNREGISTER if the callback has completed its work and should be unregistered. CONTINUE if the callback
|
||||
* should remain registered.
|
||||
*/
|
||||
public CallbackAction serverRemoved(DruidServer server);
|
||||
}
|
||||
|
||||
public static interface SegmentCallback
|
||||
{
|
||||
/**
|
||||
* Called when a segment is added to a server.
|
||||
*
|
||||
* The return value indicates if this callback has completed its work. Note that even if this callback
|
||||
* indicates that it should be unregistered, it is not possible to guarantee that this callback will not
|
||||
* get called again. There is a race condition between when this callback runs and other events that can cause
|
||||
* the callback to be queued for running. Thus, callbacks shouldn't assume that they will not get called
|
||||
* again after they are done. The contract is that the callback will eventually be unregistered, enforcing
|
||||
* a happens-before relationship is not part of the contract.
|
||||
*
|
||||
* @param server The server that added a segment
|
||||
* @param segment The segment that was added
|
||||
* @return UNREGISTER if the callback has completed its work and should be unregistered. CONTINUE if the callback
|
||||
* should remain registered.
|
||||
*/
|
||||
public CallbackAction segmentAdded(DruidServer server, DataSegment segment);
|
||||
|
||||
/**
|
||||
* Called when a segment is removed from a server.
|
||||
*
|
||||
* The return value indicates if this callback has completed its work. Note that even if this callback
|
||||
* indicates that it should be unregistered, it is not possible to guarantee that this callback will not
|
||||
* get called again. There is a race condition between when this callback runs and other events that can cause
|
||||
* the callback to be queued for running. Thus, callbacks shouldn't assume that they will not get called
|
||||
* again after they are done. The contract is that the callback will eventually be unregistered, enforcing
|
||||
* a happens-before relationship is not part of the contract.
|
||||
*
|
||||
* @param server The server that removed a segment
|
||||
* @param segment The segment that was removed
|
||||
* @return UNREGISTER if the callback has completed its work and should be unregistered. CONTINUE if the callback
|
||||
* should remain registered.
|
||||
*/
|
||||
public CallbackAction segmentRemoved(DruidServer server, DataSegment segment);
|
||||
}
|
||||
|
||||
public static abstract class BaseSegmentCallback implements SegmentCallback
|
||||
{
|
||||
@Override
|
||||
public CallbackAction segmentAdded(DruidServer server, DataSegment segment)
|
||||
{
|
||||
return CallbackAction.CONTINUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CallbackAction segmentRemoved(DruidServer server, DataSegment segment)
|
||||
{
|
||||
return CallbackAction.CONTINUE;
|
||||
}
|
||||
}
|
||||
}
|
387
client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java
Normal file
387
client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java
Normal file
@ -0,0 +1,387 @@
|
||||
package com.metamx.druid.client;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.HashBasedTable;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Table;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.phonebook.BasePhoneBook;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
import org.I0Itec.zkclient.DataUpdater;
|
||||
import org.I0Itec.zkclient.IZkChildListener;
|
||||
import org.I0Itec.zkclient.ZkClient;
|
||||
import org.I0Itec.zkclient.exception.ZkNoNodeException;
|
||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ZKPhoneBook extends BasePhoneBook
|
||||
{
|
||||
private static final Logger log = new Logger(ZKPhoneBook.class);
|
||||
|
||||
public ZKPhoneBook(
|
||||
final ObjectMapper jsonMapper,
|
||||
final ZkClient zkClient,
|
||||
final Executor peonExecutor
|
||||
)
|
||||
{
|
||||
super(
|
||||
new InternalPhoneBook(jsonMapper, zkClient, peonExecutor)
|
||||
);
|
||||
}
|
||||
|
||||
private static class InternalPhoneBook implements PhoneBook
|
||||
{
|
||||
private static final Joiner JOINER = Joiner.on("/");
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final Table<String, PhoneBookPeon, IZkChildListener> listeners;
|
||||
private final Table<String, String, Object> announcements;
|
||||
private final Map<String, PhoneBookPeon> announcementListeners;
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final ZkClient zkClient;
|
||||
private final Executor exec;
|
||||
|
||||
public InternalPhoneBook(
|
||||
ObjectMapper jsonMapper,
|
||||
ZkClient zkClient,
|
||||
Executor exec
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.zkClient = zkClient;
|
||||
this.exec = exec;
|
||||
listeners = HashBasedTable.create();
|
||||
announcements = HashBasedTable.create();
|
||||
announcementListeners = Maps.newHashMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
synchronized (lock) {
|
||||
for (Map.Entry<String, PhoneBookPeon> entry : announcementListeners.entrySet()) {
|
||||
unregisterListener(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
||||
for (Table.Cell<String, String, Object> cell : announcements.cellSet()) {
|
||||
unannounce(cell.getRowKey(), cell.getColumnKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStarted()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void announce(final String serviceName, String nodeName, T properties)
|
||||
{
|
||||
if (!zkClient.exists(serviceName)) {
|
||||
zkClient.createPersistent(serviceName, true);
|
||||
}
|
||||
|
||||
try {
|
||||
synchronized (lock) {
|
||||
zkClient.createEphemeral(
|
||||
getPath(serviceName, nodeName),
|
||||
jsonMapper.writeValueAsString(properties)
|
||||
);
|
||||
|
||||
PhoneBookPeon peon = announcementListeners.get(serviceName);
|
||||
|
||||
if (peon == null) {
|
||||
peon = new PhoneBookPeon<Object>()
|
||||
{
|
||||
@Override
|
||||
public Class<Object> getObjectClazz()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newEntry(String name, Object properties)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void entryRemoved(String name)
|
||||
{
|
||||
synchronized (lock) {
|
||||
Object propertyMap = announcements.get(serviceName, name);
|
||||
if (propertyMap != null) {
|
||||
log.info("entry[%s/%s] was removed but I'm in charge of it, reinstating.", serviceName, name);
|
||||
|
||||
String path = getPath(serviceName, name);
|
||||
try {
|
||||
zkClient.createEphemeral(
|
||||
path,
|
||||
jsonMapper.writeValueAsString(propertyMap)
|
||||
);
|
||||
}
|
||||
catch (ZkNodeExistsException e) {
|
||||
log.info("Thought that [%s] didn't exist, but it did?", path);
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.error(e, "Exception thrown when recreating node[%s].", path);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
announcementListeners.put(serviceName, peon);
|
||||
registerListener(serviceName, peon);
|
||||
}
|
||||
|
||||
announcements.put(serviceName, nodeName, properties);
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unannounce(String serviceName, String nodeName)
|
||||
{
|
||||
synchronized (lock) {
|
||||
Object announcementMap = announcements.remove(serviceName, nodeName);
|
||||
Map<String, Object> storedProperties = lookup(combineParts(Arrays.asList(serviceName, nodeName)), Map.class);
|
||||
|
||||
if (announcementMap == null || storedProperties == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
//Hack to compute equality because jsonMapper doesn't actually give me a Map<String, String> :(
|
||||
boolean areEqual = false;
|
||||
try {
|
||||
areEqual = storedProperties.equals(
|
||||
jsonMapper.readValue(jsonMapper.writeValueAsString(announcementMap), Map.class)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
log.debug("equal?[%s]: announcementMap[%s], storedProperties[%s].", areEqual, announcementMap, storedProperties);
|
||||
if (areEqual) {
|
||||
zkClient.delete(getPath(serviceName, nodeName));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T lookup(String path, Class<? extends T> clazz)
|
||||
{
|
||||
final String nodeContent;
|
||||
try {
|
||||
nodeContent = zkClient.readData(path).toString();
|
||||
}
|
||||
catch (ZkNoNodeException e) {
|
||||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
return jsonMapper.readValue(nodeContent, clazz);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void post(String serviceName, String nodeName, final T properties)
|
||||
{
|
||||
if (!zkClient.exists(serviceName)) {
|
||||
zkClient.createPersistent(serviceName);
|
||||
}
|
||||
|
||||
final String path = getPath(serviceName, nodeName);
|
||||
if (zkClient.exists(path)) {
|
||||
zkClient.updateDataSerialized(
|
||||
path,
|
||||
new DataUpdater<Object>()
|
||||
{
|
||||
@Override
|
||||
public Object update(Object currentData)
|
||||
{
|
||||
try {
|
||||
return jsonMapper.writeValueAsString(properties);
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.error(e, "Exception when updating value of [%s]. Using currentData.", path);
|
||||
return currentData;
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
try {
|
||||
zkClient.createPersistent(path, jsonMapper.writeValueAsString(properties));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean unpost(String serviceName, String nodeName)
|
||||
{
|
||||
return zkClient.delete(getPath(serviceName, nodeName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void postEphemeral(final String serviceName, String nodeName, T properties)
|
||||
{
|
||||
if (!zkClient.exists(serviceName)) {
|
||||
zkClient.createPersistent(serviceName, true);
|
||||
}
|
||||
|
||||
try {
|
||||
zkClient.createEphemeral(
|
||||
getPath(serviceName, nodeName),
|
||||
jsonMapper.writeValueAsString(properties)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> void registerListener(final String serviceName, final PhoneBookPeon<T> peon)
|
||||
{
|
||||
final Set<String> currChildren = Sets.newHashSet();
|
||||
|
||||
IZkChildListener peonListener = new IZkChildListener()
|
||||
{
|
||||
@Override
|
||||
public void handleChildChange(final String parentPath, final List<String> incomingChildren) throws Exception
|
||||
{
|
||||
exec.execute(
|
||||
new InternalPhoneBook.UpdatingRunnable<T>(
|
||||
parentPath,
|
||||
currChildren,
|
||||
(incomingChildren == null ? Sets.<String>newHashSet() : Sets.<String>newHashSet(incomingChildren)),
|
||||
peon
|
||||
)
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
zkClient.subscribeChildChanges(serviceName, peonListener);
|
||||
exec.execute(
|
||||
new UpdatingRunnable(serviceName, currChildren, Sets.newHashSet(zkClient.getChildren(serviceName)), peon)
|
||||
);
|
||||
|
||||
listeners.put(serviceName, peon, peonListener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unregisterListener(String serviceName, PhoneBookPeon peon)
|
||||
{
|
||||
IZkChildListener peonListener = listeners.get(serviceName, peon);
|
||||
|
||||
zkClient.unsubscribeChildChanges(serviceName, peonListener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String combineParts(List<String> parts)
|
||||
{
|
||||
return JOINER.join(parts);
|
||||
}
|
||||
|
||||
private String getPath(String parentPath, String child)
|
||||
{
|
||||
return JOINER.join(parentPath, child);
|
||||
}
|
||||
|
||||
private class UpdatingRunnable<T> implements Runnable
|
||||
{
|
||||
private final String serviceName;
|
||||
private final Set<String> currChildren;
|
||||
private final PhoneBookPeon<T> peon;
|
||||
private final HashSet<String> incomingChildren;
|
||||
|
||||
public UpdatingRunnable(
|
||||
String serviceName,
|
||||
Set<String> currChildren,
|
||||
final HashSet<String> incomingChildren,
|
||||
PhoneBookPeon<T> peon
|
||||
)
|
||||
{
|
||||
this.serviceName = serviceName;
|
||||
this.currChildren = currChildren;
|
||||
this.peon = peon;
|
||||
this.incomingChildren = incomingChildren;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
try {
|
||||
for (String newChild : Sets.difference(incomingChildren, currChildren)) {
|
||||
log.debug(" New child[%s], for peon[%s]", newChild, peon);
|
||||
String nodeContent;
|
||||
try {
|
||||
final String data = zkClient.readData(getPath(serviceName, newChild));
|
||||
if (data != null) {
|
||||
nodeContent = data.toString();
|
||||
}
|
||||
else {
|
||||
log.error("Ignoring path[%s] with null data", getPath(serviceName, newChild));
|
||||
continue;
|
||||
}
|
||||
}
|
||||
catch (ZkNoNodeException e) {
|
||||
log.info(
|
||||
"Got ZkNoNodeException[%s], node must have gone bye bye before this had a chance to run.",
|
||||
e.getMessage()
|
||||
);
|
||||
continue;
|
||||
}
|
||||
T nodeProperties = jsonMapper.readValue(nodeContent, peon.getObjectClazz());
|
||||
|
||||
peon.newEntry(newChild, nodeProperties);
|
||||
currChildren.add(newChild);
|
||||
}
|
||||
|
||||
// Sets.difference is lazy, so we have to materialize the difference before removing from the sets
|
||||
Set<String> setDiff = new HashSet<String>(Sets.difference(currChildren, incomingChildren));
|
||||
for (String childRemoved : setDiff) {
|
||||
log.debug(" Lost child[%s], for peon[%s]", childRemoved, peon);
|
||||
peon.entryRemoved(childRemoved);
|
||||
currChildren.remove(childRemoved);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Exception thrown, serviceName[%s].", serviceName);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
82
client/src/main/java/com/metamx/druid/client/cache/ByteCountingLRUMap.java
vendored
Normal file
82
client/src/main/java/com/metamx/druid/client/cache/ByteCountingLRUMap.java
vendored
Normal file
@ -0,0 +1,82 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
class ByteCountingLRUMap extends LinkedHashMap<ByteBuffer, byte[]>
|
||||
{
|
||||
private static final Logger log = new Logger(ByteCountingLRUMap.class);
|
||||
|
||||
private final boolean logEvictions;
|
||||
private final int logEvictionCount;
|
||||
private final long sizeInBytes;
|
||||
|
||||
private volatile long numBytes;
|
||||
private volatile long evictionCount;
|
||||
|
||||
public ByteCountingLRUMap(
|
||||
final long sizeInBytes
|
||||
)
|
||||
{
|
||||
this(16, 0, sizeInBytes);
|
||||
}
|
||||
|
||||
public ByteCountingLRUMap(
|
||||
final int initialSize,
|
||||
final int logEvictionCount,
|
||||
final long sizeInBytes
|
||||
)
|
||||
{
|
||||
super(initialSize, 0.75f, true);
|
||||
this.logEvictionCount = logEvictionCount;
|
||||
this.sizeInBytes = sizeInBytes;
|
||||
|
||||
logEvictions = logEvictionCount != 0;
|
||||
numBytes = 0;
|
||||
evictionCount = 0;
|
||||
}
|
||||
|
||||
public long getNumBytes()
|
||||
{
|
||||
return numBytes;
|
||||
}
|
||||
|
||||
public long getEvictionCount()
|
||||
{
|
||||
return evictionCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] put(ByteBuffer key, byte[] value)
|
||||
{
|
||||
numBytes += key.remaining() + value.length;
|
||||
byte[] retVal = super.put(key, value);
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean removeEldestEntry(Map.Entry<ByteBuffer, byte[]> eldest)
|
||||
{
|
||||
if (numBytes > sizeInBytes) {
|
||||
++evictionCount;
|
||||
if (logEvictions && evictionCount % logEvictionCount == 0) {
|
||||
log.info(
|
||||
"Evicting %,dth element. Size[%,d], numBytes[%,d], averageSize[%,d]",
|
||||
evictionCount,
|
||||
size(),
|
||||
numBytes,
|
||||
numBytes / size()
|
||||
);
|
||||
}
|
||||
|
||||
numBytes -= eldest.getKey().remaining() + eldest.getValue().length;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
12
client/src/main/java/com/metamx/druid/client/cache/Cache.java
vendored
Normal file
12
client/src/main/java/com/metamx/druid/client/cache/Cache.java
vendored
Normal file
@ -0,0 +1,12 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
/**
|
||||
* An interface to limit the operations that can be done on a Cache so that it is easier to reason about what
|
||||
* is actually going to be done.
|
||||
*/
|
||||
public interface Cache
|
||||
{
|
||||
public byte[] get(byte[] key);
|
||||
public byte[] put(byte[] key, byte[] value);
|
||||
public void close();
|
||||
}
|
9
client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java
vendored
Normal file
9
client/src/main/java/com/metamx/druid/client/cache/CacheBroker.java
vendored
Normal file
@ -0,0 +1,9 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheBroker
|
||||
{
|
||||
public CacheStats getStats();
|
||||
public Cache provideCache(String identifier);
|
||||
}
|
51
client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java
vendored
Normal file
51
client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java
vendored
Normal file
@ -0,0 +1,51 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.metamx.metrics.AbstractMonitor;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class CacheMonitor extends AbstractMonitor
|
||||
{
|
||||
private final CacheBroker cacheBroker;
|
||||
|
||||
private volatile CacheStats prevCacheStats = null;
|
||||
|
||||
public CacheMonitor(
|
||||
CacheBroker cacheBroker
|
||||
)
|
||||
{
|
||||
this.cacheBroker = cacheBroker;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
final CacheStats currCacheStats = cacheBroker.getStats();
|
||||
final CacheStats deltaCacheStats = currCacheStats.delta(prevCacheStats);
|
||||
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
emitStats(emitter, "cache/delta", deltaCacheStats, builder);
|
||||
emitStats(emitter, "cache/total", currCacheStats, builder);
|
||||
|
||||
prevCacheStats = currCacheStats;
|
||||
return true;
|
||||
}
|
||||
|
||||
private void emitStats(
|
||||
ServiceEmitter emitter,
|
||||
final String metricPrefix,
|
||||
CacheStats cacheStats,
|
||||
ServiceMetricEvent.Builder builder
|
||||
)
|
||||
{
|
||||
emitter.emit(builder.build(String.format("%s/numEntries", metricPrefix), cacheStats.getNumEntries()));
|
||||
emitter.emit(builder.build(String.format("%s/sizeBytes", metricPrefix), cacheStats.getSizeInBytes()));
|
||||
emitter.emit(builder.build(String.format("%s/hits", metricPrefix), cacheStats.getNumHits()));
|
||||
emitter.emit(builder.build(String.format("%s/misses", metricPrefix), cacheStats.getNumMisses()));
|
||||
emitter.emit(builder.build(String.format("%s/evictions", metricPrefix), cacheStats.getNumEvictions()));
|
||||
emitter.emit(builder.build(String.format("%s/hitRate", metricPrefix), cacheStats.hitRate()));
|
||||
emitter.emit(builder.build(String.format("%s/averageBytes", metricPrefix), cacheStats.averageBytes()));
|
||||
}
|
||||
}
|
9
client/src/main/java/com/metamx/druid/client/cache/CacheSerde.java
vendored
Normal file
9
client/src/main/java/com/metamx/druid/client/cache/CacheSerde.java
vendored
Normal file
@ -0,0 +1,9 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheSerde<V>
|
||||
{
|
||||
public byte[] serialize(V object);
|
||||
public V deserialize(byte[] bytes);
|
||||
}
|
82
client/src/main/java/com/metamx/druid/client/cache/CacheStats.java
vendored
Normal file
82
client/src/main/java/com/metamx/druid/client/cache/CacheStats.java
vendored
Normal file
@ -0,0 +1,82 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class CacheStats
|
||||
{
|
||||
private final long numHits;
|
||||
private final long numMisses;
|
||||
private final long size;
|
||||
private final long sizeInBytes;
|
||||
private final long numEvictions;
|
||||
|
||||
public CacheStats(
|
||||
long numHits,
|
||||
long numMisses,
|
||||
long size,
|
||||
long sizeInBytes,
|
||||
long numEvictions
|
||||
)
|
||||
{
|
||||
this.numHits = numHits;
|
||||
this.numMisses = numMisses;
|
||||
this.size = size;
|
||||
this.sizeInBytes = sizeInBytes;
|
||||
this.numEvictions = numEvictions;
|
||||
}
|
||||
|
||||
public long getNumHits()
|
||||
{
|
||||
return numHits;
|
||||
}
|
||||
|
||||
public long getNumMisses()
|
||||
{
|
||||
return numMisses;
|
||||
}
|
||||
|
||||
public long getNumEntries()
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
public long getSizeInBytes()
|
||||
{
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
public long getNumEvictions()
|
||||
{
|
||||
return numEvictions;
|
||||
}
|
||||
|
||||
public long numLookups()
|
||||
{
|
||||
return numHits + numMisses;
|
||||
}
|
||||
|
||||
public double hitRate()
|
||||
{
|
||||
long lookups = numLookups();
|
||||
return lookups == 0 ? 0 : numHits / (double) lookups;
|
||||
}
|
||||
|
||||
public long averageBytes()
|
||||
{
|
||||
return size == 0 ? 0 : sizeInBytes / size;
|
||||
}
|
||||
|
||||
public CacheStats delta(CacheStats oldStats)
|
||||
{
|
||||
if (oldStats == null) {
|
||||
return this;
|
||||
}
|
||||
return new CacheStats(
|
||||
numHits - oldStats.numHits,
|
||||
numMisses - oldStats.numMisses,
|
||||
size - oldStats.size,
|
||||
sizeInBytes - oldStats.sizeInBytes,
|
||||
numEvictions - oldStats.numEvictions
|
||||
);
|
||||
}
|
||||
}
|
144
client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java
vendored
Normal file
144
client/src/main/java/com/metamx/druid/client/cache/MapCacheBroker.java
vendored
Normal file
@ -0,0 +1,144 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.ISE;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MapCacheBroker implements CacheBroker
|
||||
{
|
||||
private final Map<ByteBuffer, byte[]> baseMap;
|
||||
private final ByteCountingLRUMap byteCountingLRUMap;
|
||||
|
||||
private final Map<String, Cache> cacheCache;
|
||||
private final AtomicInteger ids;
|
||||
|
||||
private final Object clearLock = new Object();
|
||||
|
||||
private final AtomicLong hitCount = new AtomicLong(0);
|
||||
private final AtomicLong missCount = new AtomicLong(0);
|
||||
|
||||
public static CacheBroker create(final MapCacheBrokerConfig config)
|
||||
{
|
||||
return new MapCacheBroker(
|
||||
new ByteCountingLRUMap(
|
||||
config.getInitialSize(),
|
||||
config.getLogEvictionCount(),
|
||||
config.getSizeInBytes()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
MapCacheBroker(
|
||||
ByteCountingLRUMap byteCountingLRUMap
|
||||
)
|
||||
{
|
||||
this.byteCountingLRUMap = byteCountingLRUMap;
|
||||
|
||||
this.baseMap = Collections.synchronizedMap(byteCountingLRUMap);
|
||||
|
||||
cacheCache = Maps.newHashMap();
|
||||
ids = new AtomicInteger();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public CacheStats getStats()
|
||||
{
|
||||
return new CacheStats(
|
||||
hitCount.get(),
|
||||
missCount.get(),
|
||||
byteCountingLRUMap.size(),
|
||||
byteCountingLRUMap.getNumBytes(),
|
||||
byteCountingLRUMap.getEvictionCount()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cache provideCache(final String identifier)
|
||||
{
|
||||
synchronized (cacheCache) {
|
||||
final Cache cachedCache = cacheCache.get(identifier);
|
||||
if (cachedCache != null) {
|
||||
return cachedCache;
|
||||
}
|
||||
|
||||
final byte[] myIdBytes = Ints.toByteArray(ids.getAndIncrement());
|
||||
|
||||
final Cache theCache = new Cache()
|
||||
{
|
||||
volatile boolean open = true;
|
||||
|
||||
@Override
|
||||
public byte[] get(byte[] key)
|
||||
{
|
||||
if (open) {
|
||||
final byte[] retVal = baseMap.get(computeKey(key));
|
||||
if (retVal == null) {
|
||||
missCount.incrementAndGet();
|
||||
} else {
|
||||
hitCount.incrementAndGet();
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
throw new ISE("Cache for identifier[%s] is closed.", identifier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] put(byte[] key, byte[] value)
|
||||
{
|
||||
synchronized (clearLock) {
|
||||
if (open) {
|
||||
return baseMap.put(computeKey(key), value);
|
||||
}
|
||||
}
|
||||
throw new ISE("Cache for identifier[%s] is closed.", identifier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
synchronized (cacheCache) {
|
||||
cacheCache.remove(identifier);
|
||||
}
|
||||
synchronized (clearLock) {
|
||||
if (open) {
|
||||
open = false;
|
||||
|
||||
Iterator<ByteBuffer> iter = baseMap.keySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
ByteBuffer next = iter.next();
|
||||
|
||||
if (next.get(0) == myIdBytes[0]
|
||||
&& next.get(1) == myIdBytes[1]
|
||||
&& next.get(2) == myIdBytes[2]
|
||||
&& next.get(3) == myIdBytes[3]) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer computeKey(byte[] key)
|
||||
{
|
||||
final ByteBuffer retVal = ByteBuffer.allocate(key.length + 4).put(myIdBytes).put(key);
|
||||
retVal.rewind();
|
||||
return retVal;
|
||||
}
|
||||
};
|
||||
|
||||
cacheCache.put(identifier, theCache);
|
||||
|
||||
return theCache;
|
||||
}
|
||||
}
|
||||
}
|
25
client/src/main/java/com/metamx/druid/client/cache/MapCacheBrokerConfig.java
vendored
Normal file
25
client/src/main/java/com/metamx/druid/client/cache/MapCacheBrokerConfig.java
vendored
Normal file
@ -0,0 +1,25 @@
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class MapCacheBrokerConfig
|
||||
{
|
||||
@Config("${prefix}.sizeInBytes")
|
||||
@Default("0")
|
||||
public abstract long getSizeInBytes();
|
||||
|
||||
@Config("${prefix}.initialSize")
|
||||
public int getInitialSize()
|
||||
{
|
||||
return 500000;
|
||||
}
|
||||
|
||||
@Config("${prefix}.logEvictionCount")
|
||||
public int getLogEvictionCount()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
package com.metamx.druid.client.selector;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.client.DruidServer;
|
||||
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ServerSelector
|
||||
{
|
||||
private static final Random random = new Random();
|
||||
|
||||
private final LinkedHashSet<DruidServer> servers = Sets.newLinkedHashSet();
|
||||
private final DataSegment segment;
|
||||
|
||||
public ServerSelector(
|
||||
DataSegment segment
|
||||
)
|
||||
{
|
||||
this.segment = segment;
|
||||
}
|
||||
|
||||
public DataSegment getSegment()
|
||||
{
|
||||
return segment;
|
||||
}
|
||||
|
||||
public void addServer(
|
||||
DruidServer server
|
||||
)
|
||||
{
|
||||
synchronized (this) {
|
||||
servers.add(server);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean removeServer(DruidServer server)
|
||||
{
|
||||
synchronized (this) {
|
||||
return servers.remove(server);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isEmpty()
|
||||
{
|
||||
synchronized (this) {
|
||||
return servers.isEmpty();
|
||||
}
|
||||
}
|
||||
|
||||
public DruidServer pick()
|
||||
{
|
||||
synchronized (this) {
|
||||
final int size = servers.size();
|
||||
switch (size) {
|
||||
case 0: return null;
|
||||
case 1: return servers.iterator().next();
|
||||
default: return Iterables.get(servers, random.nextInt(size));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
185
client/src/main/java/com/metamx/druid/http/BrokerMain.java
Normal file
185
client/src/main/java/com/metamx/druid/http/BrokerMain.java
Normal file
@ -0,0 +1,185 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceFilter;
|
||||
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
||||
import com.metamx.common.config.Config;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.CachingClusteredClient;
|
||||
import com.metamx.druid.client.ClientConfig;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.client.ClientSideServerView;
|
||||
import com.metamx.druid.client.cache.CacheBroker;
|
||||
import com.metamx.druid.client.cache.CacheMonitor;
|
||||
import com.metamx.druid.client.cache.MapCacheBroker;
|
||||
import com.metamx.druid.client.cache.MapCacheBrokerConfig;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServerConfig;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.initialization.ZkClientConfig;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.log.LogLevelAdjuster;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.druid.query.ReflectionQueryToolChestWarehouse;
|
||||
import com.metamx.emitter.core.Emitters;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import com.metamx.http.client.HttpClientConfig;
|
||||
import com.metamx.http.client.HttpClientInit;
|
||||
import com.metamx.metrics.JvmMonitor;
|
||||
import com.metamx.metrics.Monitor;
|
||||
import com.metamx.metrics.MonitorScheduler;
|
||||
import com.metamx.metrics.MonitorSchedulerConfig;
|
||||
import com.metamx.metrics.SysMonitor;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
import com.netflix.curator.x.discovery.ServiceDiscovery;
|
||||
import org.I0Itec.zkclient.ZkClient;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.smile.SmileFactory;
|
||||
import org.mortbay.jetty.Server;
|
||||
import org.mortbay.jetty.servlet.Context;
|
||||
import org.mortbay.jetty.servlet.ServletHolder;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
||||
public class BrokerMain
|
||||
{
|
||||
private static final Logger log = new Logger(BrokerMain.class);
|
||||
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
LogLevelAdjuster.register();
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
final ObjectMapper smileMapper = new DefaultObjectMapper(new SmileFactory());
|
||||
smileMapper.getJsonFactory().setCodec(smileMapper);
|
||||
|
||||
final Properties props = Initialization.loadProperties();
|
||||
final Lifecycle lifecycle = new Lifecycle();
|
||||
final ConfigurationObjectFactory configFactory = Config.createFactory(props);
|
||||
final ZkClient zkClient = Initialization.makeZkClient(configFactory.build(ZkClientConfig.class), lifecycle);
|
||||
final PhoneBook phoneBook = Initialization.createYellowPages(
|
||||
jsonMapper, zkClient, "Client-ZKYP--%s", lifecycle
|
||||
);
|
||||
|
||||
final HttpClient httpClient = HttpClientInit.createClient(
|
||||
HttpClientConfig.builder()
|
||||
.withNumConnections(
|
||||
Integer.parseInt(props.getProperty("druid.client.http.connections"))
|
||||
)
|
||||
.build(),
|
||||
lifecycle
|
||||
);
|
||||
final HttpClient emitterHttpClient = HttpClientInit.createClient(
|
||||
HttpClientConfig.builder().withNumConnections(1).build(), lifecycle
|
||||
);
|
||||
final ServiceEmitter emitter = new ServiceEmitter(
|
||||
props.getProperty("druid.service"),
|
||||
props.getProperty("druid.host"),
|
||||
Emitters.create(props, emitterHttpClient, jsonMapper, lifecycle)
|
||||
);
|
||||
|
||||
final QueryToolChestWarehouse warehouse = new ReflectionQueryToolChestWarehouse();
|
||||
final ClientConfig clientConfig = configFactory.build(ClientConfig.class);
|
||||
final ClientSideServerView view = new ClientSideServerView(warehouse, smileMapper, httpClient);
|
||||
final ClientInventoryManager clientInventoryManager = new ClientInventoryManager(
|
||||
clientConfig.getClientInventoryManagerConfig(),
|
||||
phoneBook,
|
||||
view
|
||||
);
|
||||
lifecycle.addManagedInstance(clientInventoryManager);
|
||||
|
||||
final CacheBroker cacheBroker = MapCacheBroker.create(
|
||||
configFactory.buildWithReplacements(MapCacheBrokerConfig.class, ImmutableMap.of("prefix", "druid.bard.cache"))
|
||||
);
|
||||
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cacheBroker, smileMapper);
|
||||
lifecycle.addManagedInstance(baseClient);
|
||||
|
||||
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
||||
final MonitorScheduler monitorScheduler = new MonitorScheduler(
|
||||
configFactory.build(MonitorSchedulerConfig.class),
|
||||
globalScheduledExec,
|
||||
emitter,
|
||||
ImmutableList.<Monitor>of(
|
||||
new JvmMonitor(),
|
||||
new SysMonitor(),
|
||||
new CacheMonitor(cacheBroker)
|
||||
)
|
||||
);
|
||||
lifecycle.addManagedInstance(monitorScheduler);
|
||||
|
||||
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
||||
CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient(
|
||||
serviceDiscoveryConfig.getZkHosts(),
|
||||
lifecycle
|
||||
);
|
||||
|
||||
final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
||||
curatorFramework,
|
||||
configFactory.build(ServiceDiscoveryConfig.class),
|
||||
lifecycle
|
||||
);
|
||||
|
||||
final RequestLogger requestLogger = Initialization.makeRequestLogger(
|
||||
scheduledExecutorFactory.create(
|
||||
1,
|
||||
"RequestLogger--%d"
|
||||
),
|
||||
props
|
||||
);
|
||||
lifecycle.addManagedInstance(requestLogger);
|
||||
|
||||
final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, emitter, baseClient);
|
||||
|
||||
final Injector injector = Guice.createInjector(new ClientServletModule(texasRanger, clientInventoryManager, jsonMapper));
|
||||
final Server server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class));
|
||||
final Context root = new Context(server, "/druid/v2", Context.SESSIONS);
|
||||
|
||||
root.addServlet(new ServletHolder(new StatusServlet()), "/status");
|
||||
root.addServlet(
|
||||
new ServletHolder(new QueryServlet(jsonMapper, smileMapper, texasRanger, emitter, requestLogger)),
|
||||
"/*"
|
||||
);
|
||||
|
||||
root.addEventListener(new GuiceServletConfig(injector));
|
||||
root.addFilter(GuiceFilter.class, "/heatmap/*", 0);
|
||||
root.addFilter(GuiceFilter.class, "/datasources/*", 0);
|
||||
|
||||
try {
|
||||
lifecycle.start();
|
||||
}
|
||||
catch (Throwable t) {
|
||||
log.error(t, "Error when starting up. Failing.");
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(
|
||||
new Thread(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
log.info("Running shutdown hook");
|
||||
lifecycle.stop();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
server.start();
|
||||
server.join();
|
||||
}
|
||||
}
|
@ -0,0 +1,168 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.client.DruidDataSource;
|
||||
import com.metamx.druid.client.DruidServer;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
@Path("/datasources")
|
||||
public class ClientInfoResource
|
||||
{
|
||||
private static final int SEGMENT_HISTORY_MILLIS = 7 * 24 * 60 * 60 * 1000; // ONE WEEK
|
||||
|
||||
private ClientInventoryManager clientInventoryManager;
|
||||
|
||||
@Inject
|
||||
public ClientInfoResource(
|
||||
ClientInventoryManager clientInventoryManager
|
||||
)
|
||||
{
|
||||
this.clientInventoryManager = clientInventoryManager;
|
||||
}
|
||||
|
||||
private Map<String, DruidDataSource> updateDataSources()
|
||||
{
|
||||
final Map<String, DruidDataSource> dataSources = Maps.newHashMap();
|
||||
for (DruidServer server : clientInventoryManager.getInventory()) {
|
||||
for (DruidDataSource dataSource : server.getDataSources()) {
|
||||
dataSources.put(dataSource.getName(), dataSource);
|
||||
}
|
||||
}
|
||||
return dataSources;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Produces("application/json")
|
||||
public Iterable<String> getDataSources()
|
||||
{
|
||||
return updateDataSources().keySet();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}")
|
||||
@Produces("application/json")
|
||||
public Map<String, Object> getDatasource(
|
||||
@PathParam("dataSourceName") String dataSourceName,
|
||||
@QueryParam("interval") String interval
|
||||
)
|
||||
{
|
||||
return ImmutableMap.<String, Object>of(
|
||||
"dimensions", getDatasourceDimensions(dataSourceName, interval),
|
||||
"metrics", getDatasourceMetrics(dataSourceName, interval)
|
||||
);
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}/dimensions")
|
||||
@Produces("application/json")
|
||||
public Iterable<String> getDatasourceDimensions(
|
||||
@PathParam("dataSourceName") String dataSourceName,
|
||||
@QueryParam("interval") String interval
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = updateDataSources().get(dataSourceName);
|
||||
|
||||
Set<String> retVal = Sets.newHashSet();
|
||||
|
||||
Interval dimInterval;
|
||||
if (interval == null || interval.isEmpty()) {
|
||||
Iterator<DataSegment> iter = Lists.reverse(Lists.newArrayList(dataSource.getSegments())).iterator();
|
||||
DataSegment segment = iter.next();
|
||||
retVal.addAll(segment.getDimensions());
|
||||
|
||||
dimInterval = new Interval(
|
||||
segment.getInterval().getEnd().minus(SEGMENT_HISTORY_MILLIS),
|
||||
segment.getInterval().getEnd()
|
||||
);
|
||||
|
||||
while (iter.hasNext() && dimInterval.contains(segment.getInterval())) {
|
||||
retVal.addAll(segment.getDimensions());
|
||||
segment = iter.next();
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
dimInterval = new Interval(interval);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new IAE("Interval is not in a parseable format!");
|
||||
}
|
||||
|
||||
Iterator<DataSegment> iter = dataSource.getSegments().iterator();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
DataSegment segment = iter.next();
|
||||
if (dimInterval.contains(segment.getInterval())) {
|
||||
retVal.addAll(segment.getDimensions());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}/metrics")
|
||||
@Produces("application/json")
|
||||
public Iterable<String> getDatasourceMetrics(
|
||||
@PathParam("dataSourceName") String dataSourceName,
|
||||
@QueryParam("interval") String interval
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = updateDataSources().get(dataSourceName);
|
||||
|
||||
Set<String> retVal = Sets.newHashSet();
|
||||
|
||||
Interval dimInterval;
|
||||
if (interval == null || interval.isEmpty()) {
|
||||
Iterator<DataSegment> iter = Lists.reverse(Lists.newArrayList(dataSource.getSegments())).iterator();
|
||||
DataSegment segment = iter.next();
|
||||
retVal.addAll(segment.getMetrics());
|
||||
|
||||
dimInterval = new Interval(
|
||||
segment.getInterval().getEnd().minus(SEGMENT_HISTORY_MILLIS),
|
||||
segment.getInterval().getEnd()
|
||||
);
|
||||
|
||||
while (iter.hasNext() && dimInterval.contains(segment.getInterval())) {
|
||||
retVal.addAll(segment.getMetrics());
|
||||
segment = iter.next();
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
dimInterval = new Interval(interval);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new IAE("Interval is not in a parseable format!");
|
||||
}
|
||||
|
||||
Iterator<DataSegment> iter = dataSource.getSegments().iterator();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
DataSegment segment = iter.next();
|
||||
if (dimInterval.contains(segment.getInterval())) {
|
||||
retVal.addAll(segment.getMetrics());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
22
client/src/main/java/com/metamx/druid/http/ClientMain.java
Normal file
22
client/src/main/java/com/metamx/druid/http/ClientMain.java
Normal file
@ -0,0 +1,22 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
/**
|
||||
*/
|
||||
@Deprecated
|
||||
public class ClientMain
|
||||
{
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead.");
|
||||
System.out.println("K thx bye.");
|
||||
BrokerMain.main(args);
|
||||
}
|
||||
}
|
@ -0,0 +1,73 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.client.CachingClusteredClient;
|
||||
import com.metamx.druid.query.FinalizeResultsQueryRunner;
|
||||
import com.metamx.druid.query.MetricsEmittingQueryRunner;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChest;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientQuerySegmentWalker implements QuerySegmentWalker
|
||||
{
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final ServiceEmitter emitter;
|
||||
private final CachingClusteredClient baseClient;
|
||||
|
||||
public ClientQuerySegmentWalker(
|
||||
QueryToolChestWarehouse warehouse,
|
||||
ServiceEmitter emitter,
|
||||
CachingClusteredClient baseClient
|
||||
)
|
||||
{
|
||||
this.warehouse = warehouse;
|
||||
this.emitter = emitter;
|
||||
this.baseClient = baseClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
|
||||
{
|
||||
return makeRunner(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
|
||||
{
|
||||
return makeRunner(query);
|
||||
}
|
||||
|
||||
private <T> FinalizeResultsQueryRunner<T> makeRunner(final Query<T> query)
|
||||
{
|
||||
final QueryToolChest<T,Query<T>> toolChest = warehouse.getToolChest(query);
|
||||
return new FinalizeResultsQueryRunner<T>(
|
||||
toolChest.postMergeQueryDecoration(
|
||||
toolChest.mergeResults(
|
||||
new MetricsEmittingQueryRunner<T>(
|
||||
emitter,
|
||||
new Function<Query<T>, ServiceMetricEvent.Builder>()
|
||||
{
|
||||
@Override
|
||||
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
|
||||
{
|
||||
return toolChest.makeMetricBuilder(query);
|
||||
}
|
||||
},
|
||||
toolChest.preMergeQueryDecoration(baseClient)
|
||||
)
|
||||
)
|
||||
),
|
||||
toolChest
|
||||
);
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.inject.Provides;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.sun.jersey.guice.JerseyServletModule;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import javax.inject.Singleton;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientServletModule extends JerseyServletModule
|
||||
{
|
||||
private final QuerySegmentWalker texasRanger;
|
||||
private final ClientInventoryManager clientInventoryManager;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
public ClientServletModule(
|
||||
QuerySegmentWalker texasRanger,
|
||||
ClientInventoryManager clientInventoryManager,
|
||||
ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
this.texasRanger = texasRanger;
|
||||
this.clientInventoryManager = clientInventoryManager;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configureServlets()
|
||||
{
|
||||
bind(ClientInfoResource.class);
|
||||
bind(QuerySegmentWalker.class).toInstance(texasRanger);
|
||||
bind(ClientInventoryManager.class).toInstance(clientInventoryManager);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@Singleton
|
||||
public JacksonJsonProvider getJacksonJsonProvider()
|
||||
{
|
||||
final JacksonJsonProvider provider = new JacksonJsonProvider();
|
||||
provider.setMapper(jsonMapper);
|
||||
return provider;
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,99 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.MutableDateTime;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FileRequestLogger implements RequestLogger
|
||||
{
|
||||
private final ScheduledExecutorService exec;
|
||||
private final File baseDir;
|
||||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private volatile DateTime currentDay;
|
||||
private volatile FileWriter fileWriter;
|
||||
|
||||
public FileRequestLogger(ScheduledExecutorService exec, File baseDir)
|
||||
{
|
||||
this.exec = exec;
|
||||
this.baseDir = baseDir;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
public void start()
|
||||
{
|
||||
try {
|
||||
baseDir.mkdirs();
|
||||
|
||||
MutableDateTime mutableDateTime = new DateTime().toMutableDateTime();
|
||||
mutableDateTime.setMillisOfDay(0);
|
||||
currentDay = mutableDateTime.toDateTime();
|
||||
|
||||
fileWriter = new FileWriter(new File(baseDir, currentDay.toString("yyyy-MM-dd'.log'")), true);
|
||||
long nextDay = currentDay.plusDays(1).getMillis();
|
||||
Duration delay = new Duration(nextDay - new DateTime().getMillis());
|
||||
|
||||
ScheduledExecutors.scheduleWithFixedDelay(
|
||||
exec,
|
||||
delay,
|
||||
Duration.standardDays(1),
|
||||
new Callable<ScheduledExecutors.Signal>()
|
||||
{
|
||||
@Override
|
||||
public ScheduledExecutors.Signal call()
|
||||
{
|
||||
currentDay = currentDay.plusDays(1);
|
||||
|
||||
try {
|
||||
synchronized (lock) {
|
||||
Closeables.closeQuietly(fileWriter);
|
||||
fileWriter = new FileWriter(new File(baseDir, currentDay.toString()), true);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return ScheduledExecutors.Signal.REPEAT;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
synchronized (lock) {
|
||||
Closeables.closeQuietly(fileWriter);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void log(RequestLogLine requestLogLine) throws Exception
|
||||
{
|
||||
synchronized (lock) {
|
||||
fileWriter.write(
|
||||
String.format("%s%n", requestLogLine.getLine())
|
||||
);
|
||||
fileWriter.flush();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,24 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.servlet.GuiceServletContextListener;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class GuiceServletConfig extends GuiceServletContextListener
|
||||
{
|
||||
private final Injector injector;
|
||||
|
||||
public GuiceServletConfig(
|
||||
Injector injector
|
||||
)
|
||||
{
|
||||
this.injector = injector;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Injector getInjector()
|
||||
{
|
||||
return injector;
|
||||
}
|
||||
}
|
141
client/src/main/java/com/metamx/druid/http/QueryServlet.java
Normal file
141
client/src/main/java/com/metamx/druid/http/QueryServlet.java
Normal file
@ -0,0 +1,141 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.emitter.service.AlertEvent;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.ObjectWriter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.mortbay.jetty.Request;
|
||||
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QueryServlet extends HttpServlet
|
||||
{
|
||||
private static final Logger log = new Logger(QueryServlet.class);
|
||||
private static final Charset UTF8 = Charset.forName("UTF-8");
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final ObjectMapper smileMapper;
|
||||
private final QuerySegmentWalker texasRanger;
|
||||
private final ServiceEmitter emitter;
|
||||
private final RequestLogger requestLogger;
|
||||
|
||||
public QueryServlet(
|
||||
ObjectMapper jsonMapper,
|
||||
ObjectMapper smileMapper,
|
||||
QuerySegmentWalker texasRanger,
|
||||
ServiceEmitter emitter,
|
||||
RequestLogger requestLogger
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.smileMapper = smileMapper;
|
||||
this.texasRanger = texasRanger;
|
||||
this.emitter = emitter;
|
||||
this.requestLogger = requestLogger;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException
|
||||
{
|
||||
Query query = null;
|
||||
byte[] requestQuery = null;
|
||||
|
||||
final boolean isSmile = "application/smile".equals(req.getContentType());
|
||||
|
||||
ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper;
|
||||
ObjectWriter jsonWriter = req.getParameter("pretty") == null
|
||||
? objectMapper.writer()
|
||||
: objectMapper.writerWithDefaultPrettyPrinter();
|
||||
OutputStream out = null;
|
||||
|
||||
try {
|
||||
requestQuery = ByteStreams.toByteArray(req.getInputStream());
|
||||
query = objectMapper.readValue(requestQuery, Query.class);
|
||||
|
||||
requestLogger.log(
|
||||
new RequestLogLine(new DateTime(), req.getRemoteAddr(), query)
|
||||
);
|
||||
|
||||
Sequence<?> results = query.run(texasRanger);
|
||||
|
||||
if (results == null) {
|
||||
results = Sequences.empty();
|
||||
}
|
||||
|
||||
resp.setStatus(200);
|
||||
resp.setContentType("application/x-javascript");
|
||||
|
||||
out = resp.getOutputStream();
|
||||
jsonWriter.writeValue(out, results);
|
||||
|
||||
long requestTime = System.currentTimeMillis() - ((Request) req).getTimeStamp();
|
||||
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder()
|
||||
.setUser2(query.getDataSource())
|
||||
//.setUser3(originatorType)
|
||||
.setUser4(query.getType())
|
||||
.setUser5(query.getIntervals().get(0).toString())
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
//.setUser8(originatorId)
|
||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
|
||||
.build("request/time", requestTime)
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
final String queryString =
|
||||
query == null
|
||||
? (isSmile ? "smile_unknown" : new String(requestQuery, Charsets.UTF_8))
|
||||
: query.toString();
|
||||
|
||||
log.warn(e, "Exception occurred on request [%s]", queryString);
|
||||
|
||||
if (!resp.isCommitted()) {
|
||||
resp.setStatus(500);
|
||||
resp.resetBuffer();
|
||||
|
||||
if (out == null) {
|
||||
out = resp.getOutputStream();
|
||||
}
|
||||
|
||||
out.write((e.getMessage() == null) ? "Exception null".getBytes(UTF8) : e.getMessage().getBytes(UTF8));
|
||||
out.write("\n".getBytes(UTF8));
|
||||
}
|
||||
|
||||
resp.flushBuffer();
|
||||
|
||||
emitter.emit(
|
||||
new AlertEvent.Builder().build(
|
||||
"Exception handling request",
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("exception", e.toString())
|
||||
.put("query", queryString)
|
||||
.build()
|
||||
)
|
||||
);
|
||||
}
|
||||
finally {
|
||||
resp.flushBuffer();
|
||||
Closeables.closeQuietly(out);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.metamx.druid.Query;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
public class RequestLogLine
|
||||
{
|
||||
private static final Joiner JOINER = Joiner.on("\t");
|
||||
|
||||
private final DateTime timestamp;
|
||||
private final String remoteAddr;
|
||||
private final Query query;
|
||||
|
||||
public RequestLogLine(DateTime timestamp, String remoteAddr, Query query)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.remoteAddr = remoteAddr;
|
||||
this.query = query;
|
||||
}
|
||||
|
||||
public String getLine()
|
||||
{
|
||||
return JOINER.join(
|
||||
Arrays.asList(
|
||||
timestamp,
|
||||
remoteAddr,
|
||||
query
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
@ -0,0 +1,8 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface RequestLogger
|
||||
{
|
||||
public void log(RequestLogLine requestLogLine) throws Exception;
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
package com.metamx.druid.http;
|
||||
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.PrintWriter;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class StatusServlet extends HttpServlet
|
||||
{
|
||||
@Override
|
||||
protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException
|
||||
{
|
||||
ByteArrayOutputStream retVal = new ByteArrayOutputStream();
|
||||
PrintWriter out = new PrintWriter(new OutputStreamWriter(retVal));
|
||||
|
||||
Runtime runtime = Runtime.getRuntime();
|
||||
long maxMemory = runtime.maxMemory();
|
||||
long totalMemory = runtime.totalMemory();
|
||||
long freeMemory = runtime.freeMemory();
|
||||
|
||||
out.printf("Max Memory:\t%,18d\t%1$d%n", maxMemory);
|
||||
out.printf("Total Memory:\t%,18d\t%1$d%n", totalMemory);
|
||||
out.printf("Free Memory:\t%,18d\t%1$d%n", freeMemory);
|
||||
out.printf("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory);
|
||||
|
||||
out.flush();
|
||||
|
||||
resp.setStatus(HttpServletResponse.SC_OK);
|
||||
resp.setContentType("text/plain");
|
||||
resp.getOutputStream().write(retVal.toByteArray());
|
||||
}
|
||||
}
|
@ -0,0 +1,265 @@
|
||||
package com.metamx.druid.initialization;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.ZKPhoneBook;
|
||||
import com.metamx.druid.http.FileRequestLogger;
|
||||
import com.metamx.druid.http.RequestLogger;
|
||||
import com.metamx.druid.zk.StringZkSerializer;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
import com.netflix.curator.framework.CuratorFrameworkFactory;
|
||||
import com.netflix.curator.retry.ExponentialBackoffRetry;
|
||||
import com.netflix.curator.x.discovery.ServiceDiscovery;
|
||||
import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
|
||||
import com.netflix.curator.x.discovery.ServiceInstance;
|
||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||
import org.I0Itec.zkclient.ZkClient;
|
||||
import org.I0Itec.zkclient.ZkConnection;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.mortbay.jetty.Connector;
|
||||
import org.mortbay.jetty.Server;
|
||||
import org.mortbay.jetty.nio.SelectChannelConnector;
|
||||
import org.mortbay.thread.QueuedThreadPool;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class Initialization
|
||||
{
|
||||
private static final Logger log = new Logger(Initialization.class);
|
||||
|
||||
private static volatile Properties props = null;
|
||||
|
||||
public static ZkClient makeZkClient(ZkClientConfig config, Lifecycle lifecycle)
|
||||
{
|
||||
final ZkClient retVal = new ZkClient(
|
||||
new ZkConnection(config.getZkHosts()),
|
||||
config.getConnectionTimeout(),
|
||||
new StringZkSerializer()
|
||||
);
|
||||
|
||||
lifecycle.addHandler(
|
||||
new Lifecycle.Handler()
|
||||
{
|
||||
@Override
|
||||
public void start() throws Exception
|
||||
{
|
||||
retVal.waitUntilConnected();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
retVal.close();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public static ZKPhoneBook createYellowPages(
|
||||
ObjectMapper jsonMapper, ZkClient zkClient, String threadNameFormat, Lifecycle lifecycle
|
||||
)
|
||||
{
|
||||
return lifecycle.addManagedInstance(
|
||||
new ZKPhoneBook(
|
||||
jsonMapper,
|
||||
zkClient,
|
||||
Executors.newSingleThreadExecutor(
|
||||
new ThreadFactoryBuilder()
|
||||
.setDaemon(true)
|
||||
.setNameFormat(threadNameFormat)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static Properties loadProperties()
|
||||
{
|
||||
if (props != null) {
|
||||
return props;
|
||||
}
|
||||
|
||||
Properties loadedProps = null;
|
||||
final InputStream stream = ClassLoader.getSystemResourceAsStream("runtime.properties");
|
||||
if (stream == null) {
|
||||
log.info("runtime.properties didn't exist as a resource, loading system properties instead.");
|
||||
loadedProps = System.getProperties();
|
||||
} else {
|
||||
log.info("Loading properties from runtime.properties.");
|
||||
try {
|
||||
loadedProps = new Properties();
|
||||
try {
|
||||
loadedProps.load(stream);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(stream);
|
||||
}
|
||||
}
|
||||
|
||||
for (String prop : loadedProps.stringPropertyNames()) {
|
||||
log.info("Loaded Property[%s] as [%s]", prop, loadedProps.getProperty(prop));
|
||||
}
|
||||
|
||||
props = loadedProps;
|
||||
|
||||
return loadedProps;
|
||||
}
|
||||
|
||||
public static Server makeJettyServer(ServerConfig config)
|
||||
{
|
||||
final QueuedThreadPool threadPool = new QueuedThreadPool();
|
||||
threadPool.setMinThreads(config.getNumThreads());
|
||||
threadPool.setMaxThreads(config.getNumThreads());
|
||||
|
||||
final Server server = new Server();
|
||||
server.setThreadPool(threadPool);
|
||||
|
||||
SelectChannelConnector connector = new SelectChannelConnector();
|
||||
connector.setPort(config.getPort());
|
||||
connector.setMaxIdleTime(config.getMaxIdleTimeMillis());
|
||||
connector.setStatsOn(true);
|
||||
|
||||
server.setConnectors(new Connector[]{connector});
|
||||
|
||||
return server;
|
||||
}
|
||||
|
||||
public static CuratorFramework makeCuratorFrameworkClient(
|
||||
String zkHosts,
|
||||
Lifecycle lifecycle
|
||||
) throws IOException
|
||||
{
|
||||
final CuratorFramework framework =
|
||||
CuratorFrameworkFactory.builder()
|
||||
.connectString(zkHosts)
|
||||
.retryPolicy(new ExponentialBackoffRetry(1000, 30))
|
||||
.build();
|
||||
|
||||
lifecycle.addHandler(
|
||||
new Lifecycle.Handler()
|
||||
{
|
||||
@Override
|
||||
public void start() throws Exception
|
||||
{
|
||||
framework.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
framework.close();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return framework;
|
||||
}
|
||||
|
||||
public static ServiceDiscovery makeServiceDiscoveryClient(
|
||||
CuratorFramework discoveryClient,
|
||||
ServiceDiscoveryConfig config,
|
||||
Lifecycle lifecycle
|
||||
)
|
||||
throws Exception
|
||||
{
|
||||
final ServiceInstance serviceInstance =
|
||||
ServiceInstance.builder()
|
||||
.name(config.getServiceName().replace('/', ':'))
|
||||
.port(config.getPort())
|
||||
.build();
|
||||
final ServiceDiscovery serviceDiscovery =
|
||||
ServiceDiscoveryBuilder.builder(Void.class)
|
||||
.basePath(config.getDiscoveryPath())
|
||||
.client(discoveryClient)
|
||||
.thisInstance(serviceInstance)
|
||||
.build();
|
||||
|
||||
lifecycle.addHandler(
|
||||
new Lifecycle.Handler()
|
||||
{
|
||||
@Override
|
||||
public void start() throws Exception
|
||||
{
|
||||
serviceDiscovery.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
try {
|
||||
serviceDiscovery.close();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return serviceDiscovery;
|
||||
}
|
||||
|
||||
public static ServiceProvider makeServiceProvider(
|
||||
String serviceName,
|
||||
ServiceDiscovery serviceDiscovery,
|
||||
Lifecycle lifecycle
|
||||
)
|
||||
{
|
||||
final ServiceProvider serviceProvider = serviceDiscovery.serviceProviderBuilder()
|
||||
.serviceName(serviceName)
|
||||
.build();
|
||||
|
||||
lifecycle.addHandler(
|
||||
new Lifecycle.Handler()
|
||||
{
|
||||
@Override
|
||||
public void start() throws Exception
|
||||
{
|
||||
serviceProvider.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop()
|
||||
{
|
||||
try {
|
||||
serviceProvider.close();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return serviceProvider;
|
||||
}
|
||||
|
||||
public static RequestLogger makeRequestLogger(ScheduledExecutorService exec, Properties props) throws IOException
|
||||
{
|
||||
final String property = "druid.request.logging.dir";
|
||||
final String loggingDir = props.getProperty(property);
|
||||
|
||||
if (loggingDir == null) {
|
||||
throw new ISE("property[%s] not set.", property);
|
||||
}
|
||||
|
||||
return new FileRequestLogger(exec, new File(loggingDir));
|
||||
}
|
||||
}
|
@ -0,0 +1,22 @@
|
||||
package com.metamx.druid.initialization;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ServerConfig
|
||||
{
|
||||
@Config("druid.port")
|
||||
public abstract int getPort();
|
||||
|
||||
@Config("druid.http.numThreads")
|
||||
@Default("10")
|
||||
public abstract int getNumThreads();
|
||||
|
||||
@Config("druid.http.maxIdleTimeMillis")
|
||||
public int getMaxIdleTimeMillis()
|
||||
{
|
||||
return 5 * 60 * 1000; // 5 minutes
|
||||
}
|
||||
}
|
@ -0,0 +1,20 @@
|
||||
package com.metamx.druid.initialization;
|
||||
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ServiceDiscoveryConfig
|
||||
{
|
||||
@Config("druid.service")
|
||||
public abstract String getServiceName();
|
||||
|
||||
@Config("druid.port")
|
||||
public abstract int getPort();
|
||||
|
||||
@Config("druid.zk.service.host")
|
||||
public abstract String getZkHosts();
|
||||
|
||||
@Config("druid.zk.paths.discoveryPath")
|
||||
public abstract String getDiscoveryPath();
|
||||
}
|
@ -0,0 +1,17 @@
|
||||
package com.metamx.druid.initialization;
|
||||
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ZkClientConfig
|
||||
{
|
||||
@Config("druid.zk.service.host")
|
||||
public abstract String getZkHosts();
|
||||
|
||||
@Config("druid.zk.service.connectionTimeout")
|
||||
public int getConnectionTimeout()
|
||||
{
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
@ -0,0 +1,48 @@
|
||||
package com.metamx.druid.merge;
|
||||
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientAppendQuery implements ClientMergeQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final List<DataSegment> segments;
|
||||
|
||||
@JsonCreator
|
||||
public ClientAppendQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
this.segments = segments;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public List<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ClientAppendQuery{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", segments=" + segments +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,60 @@
|
||||
package com.metamx.druid.merge;
|
||||
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClientDefaultMergeQuery implements ClientMergeQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final List<DataSegment> segments;
|
||||
private final List<AggregatorFactory> aggregators;
|
||||
|
||||
@JsonCreator
|
||||
public ClientDefaultMergeQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments,
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
this.segments = segments;
|
||||
this.aggregators = aggregators;
|
||||
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public List<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<AggregatorFactory> getAggregators()
|
||||
{
|
||||
return aggregators;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ClientDefaultMergeQuery{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", segments=" + segments +
|
||||
", aggregators=" + aggregators +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
package com.metamx.druid.merge;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public class ClientDeleteQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public ClientDeleteQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ClientDeleteQuery{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,20 @@
|
||||
package com.metamx.druid.merge;
|
||||
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class)
|
||||
@JsonSubTypes(value={
|
||||
@JsonSubTypes.Type(name="append", value=ClientAppendQuery.class)
|
||||
})
|
||||
public interface ClientMergeQuery
|
||||
{
|
||||
public String getDataSource();
|
||||
|
||||
public List<DataSegment> getSegments();
|
||||
}
|
@ -0,0 +1,86 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.guava.Yielder;
|
||||
import com.metamx.common.guava.Yielders;
|
||||
import com.metamx.common.guava.YieldingAccumulator;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||
import com.metamx.druid.result.Result;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BySegmentQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final String segmentIdentifier;
|
||||
private final DateTime timestamp;
|
||||
private final QueryRunner<T> base;
|
||||
|
||||
public BySegmentQueryRunner(
|
||||
String segmentIdentifier,
|
||||
DateTime timestamp,
|
||||
QueryRunner<T> base
|
||||
)
|
||||
{
|
||||
this.segmentIdentifier = segmentIdentifier;
|
||||
this.timestamp = timestamp;
|
||||
this.base = base;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
if (Boolean.parseBoolean(query.getContextValue("bySegment"))) {
|
||||
final Sequence<T> baseSequence = base.run(query);
|
||||
return new Sequence<T>()
|
||||
{
|
||||
@Override
|
||||
public <OutType> OutType accumulate(OutType initValue, Accumulator<OutType, T> accumulator)
|
||||
{
|
||||
List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
|
||||
|
||||
return accumulator.accumulate(
|
||||
initValue,
|
||||
(T) new Result<BySegmentResultValueClass<T>>(
|
||||
timestamp,
|
||||
new BySegmentResultValueClass<T>(
|
||||
results,
|
||||
segmentIdentifier,
|
||||
query.getIntervals().get(0).toString()
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
|
||||
{
|
||||
List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
|
||||
|
||||
final OutType retVal = accumulator.accumulate(
|
||||
initValue,
|
||||
(T) new Result<BySegmentResultValueClass<T>>(
|
||||
timestamp,
|
||||
new BySegmentResultValueClass<T>(
|
||||
results,
|
||||
segmentIdentifier,
|
||||
query.getIntervals().get(0).toString()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
return Yielders.done(retVal, null);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return base.run(query);
|
||||
}
|
||||
}
|
@ -0,0 +1,30 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final QueryRunner<T> baseRunner;
|
||||
|
||||
public BySegmentSkippingQueryRunner(
|
||||
QueryRunner<T> baseRunner
|
||||
)
|
||||
{
|
||||
this.baseRunner = baseRunner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(Query<T> query)
|
||||
{
|
||||
if (Boolean.parseBoolean(query.getContextValue("bySegment"))) {
|
||||
return baseRunner.run(query);
|
||||
}
|
||||
|
||||
return doRun(baseRunner, query);
|
||||
}
|
||||
|
||||
protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query);
|
||||
}
|
@ -0,0 +1,18 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheStrategy<T, QueryType extends Query<T>>
|
||||
{
|
||||
public byte[] computeCacheKey(QueryType query);
|
||||
|
||||
public Function<T, Object> prepareForCache();
|
||||
|
||||
public Function<Object, T> pullFromCache();
|
||||
|
||||
public Sequence<T> mergeSequences(Sequence<Sequence<T>> seqOfSequences);
|
||||
}
|
@ -0,0 +1,153 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.guava.BaseSequence;
|
||||
import com.metamx.common.guava.MergeIterable;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
/**
|
||||
* A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor.
|
||||
* <p/>
|
||||
* When using this, it is important to make sure that the list of QueryRunners provided is fully flattened.
|
||||
* If, for example, you were to pass a list of a Chained QueryRunner (A) and a non-chained QueryRunner (B). Imagine
|
||||
* A has 2 QueryRunner chained together (Aa and Ab), the fact that the Queryables are run in parallel on an
|
||||
* executor would mean that the Queryables are actually processed in the order
|
||||
* <p/>
|
||||
* A -> B -> Aa -> Ab
|
||||
* <p/>
|
||||
* That is, the two sub queryables for A would run *after* B is run, effectively meaning that the results for B
|
||||
* must be fully cached in memory before the results for Aa and Ab are computed.
|
||||
*/
|
||||
public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final Iterable<QueryRunner<T>> queryables;
|
||||
private final ExecutorService exec;
|
||||
private final Ordering<T> ordering;
|
||||
|
||||
public ChainedExecutionQueryRunner(
|
||||
ExecutorService exec,
|
||||
Ordering<T> ordering,
|
||||
QueryRunner<T>... queryables
|
||||
)
|
||||
{
|
||||
this(exec, ordering, Arrays.asList(queryables));
|
||||
}
|
||||
|
||||
public ChainedExecutionQueryRunner(
|
||||
ExecutorService exec,
|
||||
Ordering<T> ordering,
|
||||
Iterable<QueryRunner<T>> queryables
|
||||
)
|
||||
{
|
||||
this.exec = exec;
|
||||
this.ordering = ordering;
|
||||
this.queryables = Iterables.unmodifiableIterable(queryables);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
return new BaseSequence<T, Iterator<T>>(
|
||||
new BaseSequence.IteratorMaker<T, Iterator<T>>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<T> make()
|
||||
{
|
||||
// Make it a List<> to materialize all of the values (so that it will submit everything to the executor)
|
||||
List<Future<List<T>>> futures = Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
queryables,
|
||||
new Function<QueryRunner<T>, Future<List<T>>>()
|
||||
{
|
||||
@Override
|
||||
public Future<List<T>> apply(final QueryRunner<T> input)
|
||||
{
|
||||
return exec.submit(
|
||||
new Callable<List<T>>()
|
||||
{
|
||||
@Override
|
||||
public List<T> call() throws Exception
|
||||
{
|
||||
return Sequences.toList(input.run(query), Lists.<T>newArrayList());
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
return new MergeIterable<T>(
|
||||
ordering.nullsFirst(),
|
||||
Iterables.transform(
|
||||
futures,
|
||||
new Function<Future<List<T>>, Iterable<T>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<T> apply(Future<List<T>> input)
|
||||
{
|
||||
try {
|
||||
return input.get();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
)
|
||||
).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup(Iterator<T> tIterator)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private static class ThreadNamingCallable<T> implements Callable<T>
|
||||
{
|
||||
private final Callable<T> baseCallable;
|
||||
private final String name;
|
||||
|
||||
ThreadNamingCallable(
|
||||
Callable<T> baseCallable,
|
||||
String name
|
||||
)
|
||||
{
|
||||
this.baseCallable = baseCallable;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T call() throws Exception
|
||||
{
|
||||
String oldName = Thread.currentThread().getName();
|
||||
try {
|
||||
Thread.currentThread().setName(name);
|
||||
return baseCallable.call();
|
||||
}
|
||||
finally {
|
||||
Thread.currentThread().setName(oldName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ConcatQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final Sequence<QueryRunner<T>> queryRunners;
|
||||
|
||||
public ConcatQueryRunner(
|
||||
Sequence<QueryRunner<T>> queryRunners
|
||||
) {
|
||||
this.queryRunners = queryRunners;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
return Sequences.concat(
|
||||
Sequences.map(
|
||||
queryRunners,
|
||||
new Function<QueryRunner<T>, Sequence<T>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<T> apply(final QueryRunner<T> input)
|
||||
{
|
||||
return input.run(query);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
@ -0,0 +1,92 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||
import com.metamx.druid.result.Result;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final QueryRunner<T> baseRunner;
|
||||
private final QueryToolChest<T, Query<T>> toolChest;
|
||||
|
||||
public FinalizeResultsQueryRunner(
|
||||
QueryRunner<T> baseRunner,
|
||||
QueryToolChest<T, Query<T>> toolChest
|
||||
)
|
||||
{
|
||||
this.baseRunner = baseRunner;
|
||||
this.toolChest = toolChest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment"));
|
||||
final boolean shouldFinalize = Boolean.parseBoolean(query.getContextValue("finalize", "true"));
|
||||
if (shouldFinalize) {
|
||||
Function<T, T> finalizerFn;
|
||||
if (isBySegment) {
|
||||
finalizerFn = new Function<T, T>()
|
||||
{
|
||||
final Function<T, T> baseFinalizer = toolChest.makeMetricManipulatorFn(
|
||||
query,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.finalizeComputation(factory.deserialize(object));
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public T apply(@Nullable T input)
|
||||
{
|
||||
Result<BySegmentResultValueClass<T>> result = (Result<BySegmentResultValueClass<T>>) input;
|
||||
BySegmentResultValueClass<T> resultsClass = result.getValue();
|
||||
|
||||
return (T) new Result<BySegmentResultValueClass>(
|
||||
result.getTimestamp(),
|
||||
new BySegmentResultValueClass(
|
||||
Lists.transform(resultsClass.getResults(), baseFinalizer),
|
||||
resultsClass.getSegmentId(),
|
||||
resultsClass.getIntervalString()
|
||||
)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
else {
|
||||
finalizerFn = toolChest.makeMetricManipulatorFn(
|
||||
query,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.finalizeComputation(object);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
return Sequences.map(
|
||||
baseRunner.run(query.withOverriddenContext(ImmutableMap.of("finalize", "false"))),
|
||||
finalizerFn
|
||||
);
|
||||
}
|
||||
return baseRunner.run(query);
|
||||
}
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.PeriodGranularity;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final QueryRunner<T> baseRunner;
|
||||
private final Period period;
|
||||
|
||||
public IntervalChunkingQueryRunner(QueryRunner<T> baseRunner, Period period)
|
||||
{
|
||||
this.baseRunner = baseRunner;
|
||||
this.period = period;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
return Sequences.concat(
|
||||
FunctionalIterable
|
||||
.create(query.getIntervals())
|
||||
.transformCat(
|
||||
new Function<Interval, Iterable<Interval>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<Interval> apply(@Nullable Interval input)
|
||||
{
|
||||
return splitInterval(input);
|
||||
}
|
||||
}
|
||||
)
|
||||
.transform(
|
||||
new Function<Interval, Sequence<T>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<T> apply(Interval singleInterval)
|
||||
{
|
||||
return baseRunner.run(
|
||||
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval)))
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private Iterable<Interval> splitInterval(Interval interval)
|
||||
{
|
||||
if (interval.getEndMillis() == interval.getStartMillis()) {
|
||||
return Lists.newArrayList(interval);
|
||||
}
|
||||
|
||||
List<Interval> intervals = Lists.newArrayList();
|
||||
Iterator<Long> timestamps = new PeriodGranularity(period, null, null).iterable(
|
||||
interval.getStartMillis(),
|
||||
interval.getEndMillis()
|
||||
).iterator();
|
||||
|
||||
long start = Math.max(timestamps.next(), interval.getStartMillis());
|
||||
while (timestamps.hasNext()) {
|
||||
long end = timestamps.next();
|
||||
intervals.add(new Interval(start, end));
|
||||
start = end;
|
||||
}
|
||||
|
||||
if (start < interval.getEndMillis()) {
|
||||
intervals.add(new Interval(start, interval.getEndMillis()));
|
||||
}
|
||||
|
||||
return intervals;
|
||||
}
|
||||
}
|
@ -0,0 +1,10 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface MetricManipulationFn
|
||||
{
|
||||
public Object manipulate(AggregatorFactory factory, Object object);
|
||||
}
|
@ -0,0 +1,139 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Yielder;
|
||||
import com.metamx.common.guava.YieldingAccumulator;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
private final ServiceEmitter emitter;
|
||||
private final Function<Query<T>, ServiceMetricEvent.Builder> builderFn;
|
||||
private final QueryRunner<T> queryRunner;
|
||||
|
||||
public MetricsEmittingQueryRunner(
|
||||
ServiceEmitter emitter,
|
||||
Function<Query<T>, ServiceMetricEvent.Builder> builderFn,
|
||||
QueryRunner<T> queryRunner
|
||||
)
|
||||
{
|
||||
this.emitter = emitter;
|
||||
this.builderFn = builderFn;
|
||||
this.queryRunner = queryRunner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
final ServiceMetricEvent.Builder builder = builderFn.apply(query);
|
||||
|
||||
return new Sequence<T>()
|
||||
{
|
||||
@Override
|
||||
public <OutType> OutType accumulate(OutType outType, Accumulator<OutType, T> accumulator)
|
||||
{
|
||||
OutType retVal;
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
try {
|
||||
retVal = queryRunner.run(query).accumulate(outType, accumulator);
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
catch (Error e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
|
||||
emitter.emit(builder.build("query/time", timeTaken));
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(OutType initValue, YieldingAccumulator<OutType, T> accumulator)
|
||||
{
|
||||
Yielder<OutType> retVal;
|
||||
|
||||
long startTime = System.currentTimeMillis();
|
||||
try {
|
||||
retVal = queryRunner.run(query).toYielder(initValue, accumulator);
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
catch (Error e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
|
||||
return makeYielder(startTime, retVal, builder);
|
||||
}
|
||||
|
||||
private <OutType> Yielder<OutType> makeYielder(
|
||||
final long startTime,
|
||||
final Yielder<OutType> yielder,
|
||||
final ServiceMetricEvent.Builder builder
|
||||
)
|
||||
{
|
||||
return new Yielder<OutType>()
|
||||
{
|
||||
@Override
|
||||
public OutType get()
|
||||
{
|
||||
return yielder.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Yielder<OutType> next(OutType initValue)
|
||||
{
|
||||
try {
|
||||
return makeYielder(startTime, yielder.next(initValue), builder);
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
catch (Error e) {
|
||||
builder.setUser10("failed");
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDone()
|
||||
{
|
||||
return yielder.isDone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
if (!isDone() && builder.getUser10() == null) {
|
||||
builder.setUser10("short");
|
||||
}
|
||||
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
emitter.emit(builder.build("query/time", timeTaken));
|
||||
|
||||
yielder.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
@ -0,0 +1,16 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NoopQueryRunner<T> implements QueryRunner<T>
|
||||
{
|
||||
@Override
|
||||
public Sequence<T> run(Query query)
|
||||
{
|
||||
return Sequences.empty();
|
||||
}
|
||||
}
|
@ -0,0 +1,31 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QueryCacheHelper
|
||||
{
|
||||
public static byte[] computeAggregatorBytes(List<AggregatorFactory> aggregatorSpecs)
|
||||
{
|
||||
List<byte[]> cacheKeySet = Lists.newArrayListWithCapacity(aggregatorSpecs.size());
|
||||
|
||||
int totalSize = 0;
|
||||
for (AggregatorFactory spec : aggregatorSpecs) {
|
||||
final byte[] cacheKey = spec.getCacheKey();
|
||||
cacheKeySet.add(cacheKey);
|
||||
totalSize += cacheKey.length;
|
||||
}
|
||||
|
||||
ByteBuffer retVal = ByteBuffer.allocate(totalSize);
|
||||
for (byte[] bytes : cacheKeySet) {
|
||||
retVal.put(bytes);
|
||||
}
|
||||
return retVal.array();
|
||||
}
|
||||
|
||||
}
|
11
client/src/main/java/com/metamx/druid/query/QueryRunner.java
Normal file
11
client/src/main/java/com/metamx/druid/query/QueryRunner.java
Normal file
@ -0,0 +1,11 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface QueryRunner<T>
|
||||
{
|
||||
public Sequence<T> run(Query<T> query);
|
||||
}
|
@ -0,0 +1,31 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
/**
|
||||
* The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing
|
||||
* evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential
|
||||
* maintenance burden when upgrading versions.
|
||||
*/
|
||||
public interface QueryToolChest<ResultType, QueryType extends Query<ResultType>>
|
||||
{
|
||||
public QueryRunner<ResultType> mergeResults(QueryRunner<ResultType> runner);
|
||||
|
||||
/**
|
||||
* This method doesn't belong here, but it's here for now just to make it work.
|
||||
*
|
||||
* @param seqOfSequences
|
||||
* @return
|
||||
*/
|
||||
public Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
|
||||
public Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
|
||||
public TypeReference<ResultType> getResultTypeReference();
|
||||
public CacheStrategy<ResultType, QueryType> getCacheStrategy(QueryType query);
|
||||
public QueryRunner<ResultType> preMergeQueryDecoration(QueryRunner<ResultType> runner);
|
||||
public QueryRunner<ResultType> postMergeQueryDecoration(QueryRunner<ResultType> runner);
|
||||
}
|
@ -0,0 +1,10 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface QueryToolChestWarehouse
|
||||
{
|
||||
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query);
|
||||
}
|
@ -0,0 +1,56 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ReflectionLoaderThingy<T>
|
||||
{
|
||||
private static final Logger log = new Logger(ReflectionLoaderThingy.class);
|
||||
|
||||
public static <K> ReflectionLoaderThingy<K> create(Class<K> interfaceClass)
|
||||
{
|
||||
return new ReflectionLoaderThingy<K>(interfaceClass);
|
||||
}
|
||||
|
||||
Map<Class<?>, AtomicReference<T>> toolChestMap = Maps.newConcurrentMap();
|
||||
|
||||
private final Class<T> interfaceClass;
|
||||
|
||||
public ReflectionLoaderThingy(
|
||||
Class<T> interfaceClass
|
||||
)
|
||||
{
|
||||
this.interfaceClass = interfaceClass;
|
||||
}
|
||||
|
||||
public T getForObject(Object keyObject)
|
||||
{
|
||||
Class<?> clazz = keyObject.getClass();
|
||||
|
||||
AtomicReference<T> retVal = toolChestMap.get(clazz);
|
||||
|
||||
if (retVal == null) {
|
||||
String interfaceName = interfaceClass.getSimpleName();
|
||||
|
||||
AtomicReference<T> retVal1;
|
||||
try {
|
||||
final Class<?> queryToolChestClass = Class.forName(String.format("%s%s", clazz.getName(), interfaceName));
|
||||
retVal1 = new AtomicReference<T>(interfaceClass.cast(queryToolChestClass.newInstance()));
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Unable to load interface[%s] for input class[%s]", interfaceClass, clazz);
|
||||
retVal1 = new AtomicReference<T>(null);
|
||||
}
|
||||
retVal = retVal1;
|
||||
|
||||
toolChestMap.put(clazz, retVal);
|
||||
}
|
||||
|
||||
return retVal.get();
|
||||
}
|
||||
}
|
@ -0,0 +1,17 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.metamx.druid.Query;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
|
||||
{
|
||||
ReflectionLoaderThingy<QueryToolChest> loader = ReflectionLoaderThingy.create(QueryToolChest.class);
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query)
|
||||
{
|
||||
return (QueryToolChest<T, QueryType>) loader.getForObject(query);
|
||||
}
|
||||
}
|
@ -0,0 +1,29 @@
|
||||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.guava.CombiningSequence;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRunner<T>
|
||||
{
|
||||
public ResultMergeQueryRunner(
|
||||
QueryRunner<T> baseRunner
|
||||
)
|
||||
{
|
||||
super(baseRunner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query)
|
||||
{
|
||||
return CombiningSequence.create(baseRunner.run(query), makeOrdering(query), createMergeFn(query));
|
||||
}
|
||||
|
||||
protected abstract Ordering<T> makeOrdering(Query<T> query);
|
||||
|
||||
protected abstract BinaryFn<T,T,T> createMergeFn(Query<T> query);
|
||||
}
|
@ -0,0 +1,68 @@
|
||||
package com.metamx.druid.query.dimension;
|
||||
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DefaultDimensionSpec implements DimensionSpec
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x0;
|
||||
private final String dimension;
|
||||
private final String outputName;
|
||||
|
||||
@JsonCreator
|
||||
public DefaultDimensionSpec(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("outputName") String outputName
|
||||
)
|
||||
{
|
||||
this.dimension = dimension.toLowerCase();
|
||||
|
||||
// Do null check for legacy backwards compatibility, callers should be setting the value.
|
||||
this.outputName = outputName == null ? dimension : outputName;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getOutputName()
|
||||
{
|
||||
return outputName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimExtractionFn getDimExtractionFn()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] dimensionBytes = dimension.getBytes();
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(dimensionBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DefaultDimensionSpec{" +
|
||||
"dimension='" + dimension + '\'' +
|
||||
", outputName='" + outputName + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,20 @@
|
||||
package com.metamx.druid.query.dimension;
|
||||
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class),
|
||||
@JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class)
|
||||
})
|
||||
public interface DimensionSpec
|
||||
{
|
||||
public String getDimension();
|
||||
public String getOutputName();
|
||||
public DimExtractionFn getDimExtractionFn();
|
||||
public byte[] getCacheKey();
|
||||
}
|
@ -0,0 +1,76 @@
|
||||
package com.metamx.druid.query.dimension;
|
||||
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ExtractionDimensionSpec implements DimensionSpec
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
private final String dimension;
|
||||
private final DimExtractionFn dimExtractionFn;
|
||||
private final String outputName;
|
||||
|
||||
@JsonCreator
|
||||
public ExtractionDimensionSpec(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("outputName") String outputName,
|
||||
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
||||
)
|
||||
{
|
||||
this.dimension = dimension.toLowerCase();
|
||||
this.dimExtractionFn = dimExtractionFn;
|
||||
|
||||
// Do null check for backwards compatibility
|
||||
this.outputName = outputName == null ? dimension : outputName;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getOutputName()
|
||||
{
|
||||
return outputName;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public DimExtractionFn getDimExtractionFn()
|
||||
{
|
||||
return dimExtractionFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] dimensionBytes = dimension.getBytes();
|
||||
byte[] dimExtractionFnBytes = dimExtractionFn.getCacheKey();
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(dimensionBytes)
|
||||
.put(dimExtractionFnBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ExtractionDimensionSpec{" +
|
||||
"dimension='" + dimension + '\'' +
|
||||
", dimExtractionFn=" + dimExtractionFn +
|
||||
", outputName='" + outputName + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
package com.metamx.druid.query.dimension;
|
||||
|
||||
import com.metamx.common.IAE;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LegacyDimensionSpec extends DefaultDimensionSpec
|
||||
{
|
||||
private static final String convertValue(Object dimension, String name)
|
||||
{
|
||||
final String retVal;
|
||||
|
||||
if (dimension instanceof String) {
|
||||
retVal = (String) dimension;
|
||||
} else if (dimension instanceof Map) {
|
||||
retVal = (String) ((Map) dimension).get(name);
|
||||
} else {
|
||||
throw new IAE("Unknown type[%s] for dimension[%s]", dimension.getClass(), dimension);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public LegacyDimensionSpec(Object dimension)
|
||||
{
|
||||
super(convertValue(dimension, "dimension"), convertValue(dimension, "outputName"));
|
||||
}
|
||||
}
|
@ -0,0 +1,19 @@
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class),
|
||||
@JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class)
|
||||
})
|
||||
public interface DimExtractionFn
|
||||
{
|
||||
public byte[] getCacheKey();
|
||||
public String apply(String dimValue);
|
||||
}
|
@ -0,0 +1,56 @@
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class PartialDimExtractionFn implements DimExtractionFn
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x2;
|
||||
|
||||
private final String expr;
|
||||
private final Pattern pattern;
|
||||
|
||||
@JsonCreator
|
||||
public PartialDimExtractionFn(
|
||||
@JsonProperty("expr") String expr
|
||||
)
|
||||
{
|
||||
this.expr = expr;
|
||||
this.pattern = Pattern.compile(expr);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] exprBytes = expr.getBytes();
|
||||
return ByteBuffer.allocate(1 + exprBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(exprBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
Matcher matcher = pattern.matcher(dimValue);
|
||||
return matcher.find() ? dimValue : null;
|
||||
}
|
||||
|
||||
@JsonProperty("expr")
|
||||
public String getExpr()
|
||||
{
|
||||
return expr;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("regex_matches(%s)", expr);
|
||||
}
|
||||
}
|
@ -0,0 +1,56 @@
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class RegexDimExtractionFn implements DimExtractionFn
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
private final String expr;
|
||||
private final Pattern pattern;
|
||||
|
||||
@JsonCreator
|
||||
public RegexDimExtractionFn(
|
||||
@JsonProperty("expr") String expr
|
||||
)
|
||||
{
|
||||
this.expr = expr;
|
||||
this.pattern = Pattern.compile(expr);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] exprBytes = expr.getBytes();
|
||||
return ByteBuffer.allocate(1 + exprBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(exprBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
Matcher matcher = pattern.matcher(dimValue);
|
||||
return matcher.find() ? matcher.group(1) : dimValue;
|
||||
}
|
||||
|
||||
@JsonProperty("expr")
|
||||
public String getExpr()
|
||||
{
|
||||
return expr;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("regex(%s)", expr);
|
||||
}
|
||||
}
|
@ -0,0 +1,54 @@
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
import com.metamx.druid.query.search.SearchQuerySpec;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SearchQuerySpecDimExtractionFn implements DimExtractionFn
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x3;
|
||||
|
||||
private final SearchQuerySpec searchQuerySpec;
|
||||
|
||||
@JsonCreator
|
||||
public SearchQuerySpecDimExtractionFn(
|
||||
@JsonProperty("query") SearchQuerySpec searchQuerySpec
|
||||
)
|
||||
{
|
||||
this.searchQuerySpec = searchQuerySpec;
|
||||
}
|
||||
|
||||
@JsonProperty("query")
|
||||
public SearchQuerySpec getSearchQuerySpec()
|
||||
{
|
||||
return searchQuerySpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] specBytes = searchQuerySpec.getCacheKey();
|
||||
return ByteBuffer.allocate(1 + specBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(specBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
return searchQuerySpec.accept(dimValue) ? dimValue : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SearchQuerySpecDimExtractionFn{" +
|
||||
"searchQuerySpec=" + searchQuerySpec +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,79 @@
|
||||
package com.metamx.druid.query.extraction;
|
||||
|
||||
import com.ibm.icu.text.SimpleDateFormat;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.text.ParseException;
|
||||
import java.util.Date;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class TimeDimExtractionFn implements DimExtractionFn
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x0;
|
||||
|
||||
private final String timeFormat;
|
||||
private final SimpleDateFormat timeFormatter;
|
||||
private final String resultFormat;
|
||||
private final SimpleDateFormat resultFormatter;
|
||||
|
||||
@JsonCreator
|
||||
public TimeDimExtractionFn(
|
||||
@JsonProperty("timeFormat") String timeFormat,
|
||||
@JsonProperty("resultFormat") String resultFormat
|
||||
)
|
||||
{
|
||||
this.timeFormat = timeFormat;
|
||||
this.timeFormatter = new SimpleDateFormat(timeFormat);
|
||||
this.timeFormatter.setLenient(true);
|
||||
|
||||
this.resultFormat = resultFormat;
|
||||
this.resultFormatter = new SimpleDateFormat(resultFormat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] timeFormatBytes = timeFormat.getBytes();
|
||||
return ByteBuffer.allocate(1 + timeFormatBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(timeFormatBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String apply(String dimValue)
|
||||
{
|
||||
Date date;
|
||||
try {
|
||||
date = timeFormatter.parse(dimValue);
|
||||
}
|
||||
catch (ParseException e) {
|
||||
return dimValue;
|
||||
}
|
||||
return resultFormatter.format(date);
|
||||
}
|
||||
|
||||
@JsonProperty("timeFormat")
|
||||
public String getTimeFormat()
|
||||
{
|
||||
return timeFormat;
|
||||
}
|
||||
|
||||
@JsonProperty("resultFormat")
|
||||
public String getResultFormat()
|
||||
{
|
||||
return resultFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "TimeDimExtractionFn{" +
|
||||
"timeFormat='" + timeFormat + '\'' +
|
||||
", resultFormat='" + resultFormat + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
@ -0,0 +1,68 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class AndDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
private static final Joiner AND_JOINER = Joiner.on(" && ");
|
||||
|
||||
final private List<DimFilter> fields;
|
||||
|
||||
@JsonCreator
|
||||
public AndDimFilter(
|
||||
@JsonProperty("fields") List<DimFilter> fields
|
||||
)
|
||||
{
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<DimFilter> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return DimFilterCacheHelper.computeCacheKey(CACHE_TYPE_ID, fields);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
AndDimFilter that = (AndDimFilter) o;
|
||||
|
||||
if (fields != null ? !fields.equals(that.fields) : that.fields != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return fields != null ? fields.hashCode() : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("(%s)", AND_JOINER.join(fields));
|
||||
}
|
||||
}
|
@ -0,0 +1,20 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type")
|
||||
@JsonSubTypes(value={
|
||||
@JsonSubTypes.Type(name="and", value=AndDimFilter.class),
|
||||
@JsonSubTypes.Type(name="or", value=OrDimFilter.class),
|
||||
@JsonSubTypes.Type(name="not", value=NotDimFilter.class),
|
||||
@JsonSubTypes.Type(name="selector", value=SelectorDimFilter.class),
|
||||
@JsonSubTypes.Type(name="extraction", value=ExtractionDimFilter.class),
|
||||
@JsonSubTypes.Type(name="regex", value=RegexDimFilter.class)
|
||||
})
|
||||
public interface DimFilter
|
||||
{
|
||||
public byte[] getCacheKey();
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
class DimFilterCacheHelper
|
||||
{
|
||||
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
||||
{
|
||||
if (filters.size() == 1) {
|
||||
return filters.get(0).getCacheKey();
|
||||
}
|
||||
|
||||
byte[][] cacheKeys = new byte[filters.size()][];
|
||||
int totalSize = 0;
|
||||
int index = 0;
|
||||
for (DimFilter field : filters) {
|
||||
cacheKeys[index] = field.getCacheKey();
|
||||
totalSize += cacheKeys[index].length;
|
||||
++index;
|
||||
}
|
||||
|
||||
ByteBuffer retVal = ByteBuffer.allocate(1 + totalSize);
|
||||
retVal.put(cacheIdKey);
|
||||
for (byte[] cacheKey : cacheKeys) {
|
||||
retVal.put(cacheKey);
|
||||
}
|
||||
return retVal.array();
|
||||
}
|
||||
}
|
@ -0,0 +1,67 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import com.metamx.druid.query.extraction.DimExtractionFn;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ExtractionDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x4;
|
||||
|
||||
private final String dimension;
|
||||
private final String value;
|
||||
private final DimExtractionFn dimExtractionFn;
|
||||
|
||||
@JsonCreator
|
||||
public ExtractionDimFilter(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("value") String value,
|
||||
@JsonProperty("dimExtractionFn") DimExtractionFn dimExtractionFn
|
||||
)
|
||||
{
|
||||
this.dimension = dimension.toLowerCase();
|
||||
this.value = value;
|
||||
this.dimExtractionFn = dimExtractionFn;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DimExtractionFn getDimExtractionFn()
|
||||
{
|
||||
return dimExtractionFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] dimensionBytes = dimension.getBytes();
|
||||
byte[] valueBytes = value.getBytes();
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(dimensionBytes)
|
||||
.put(valueBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("%s(%s) = %s", dimExtractionFn, dimension, value);
|
||||
}
|
||||
}
|
@ -0,0 +1,16 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NoopDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = -0x4;
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return ByteBuffer.allocate(1).put(CACHE_TYPE_ID).array();
|
||||
}
|
||||
}
|
@ -0,0 +1,68 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NotDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x3;
|
||||
|
||||
final private DimFilter field;
|
||||
|
||||
@JsonCreator
|
||||
public NotDimFilter(
|
||||
@JsonProperty("field") DimFilter field
|
||||
)
|
||||
{
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
@JsonProperty("field")
|
||||
public DimFilter getField()
|
||||
{
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] subKey = field.getCacheKey();
|
||||
|
||||
return ByteBuffer.allocate(1 + subKey.length).put(CACHE_TYPE_ID).put(subKey).array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
NotDimFilter that = (NotDimFilter) o;
|
||||
|
||||
if (field != null ? !field.equals(that.field) : that.field != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return field != null ? field.hashCode() : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "!" + field;
|
||||
}
|
||||
}
|
@ -0,0 +1,68 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class OrDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x2;
|
||||
private static final Joiner OR_JOINER = Joiner.on(" || ");
|
||||
|
||||
final private List<DimFilter> fields;
|
||||
|
||||
@JsonCreator
|
||||
public OrDimFilter(
|
||||
@JsonProperty("fields") List<DimFilter> fields
|
||||
)
|
||||
{
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<DimFilter> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return DimFilterCacheHelper.computeCacheKey(CACHE_TYPE_ID, fields);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
OrDimFilter that = (OrDimFilter) o;
|
||||
|
||||
if (fields != null ? !fields.equals(that.fields) : that.fields != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return fields != null ? fields.hashCode() : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("(%s)", OR_JOINER.join(fields));
|
||||
}
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class RegexDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_ID_KEY = 0x5;
|
||||
private final String dimension;
|
||||
private final String pattern;
|
||||
|
||||
@JsonCreator
|
||||
public RegexDimFilter(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("pattern") String pattern
|
||||
)
|
||||
{
|
||||
this.dimension = dimension;
|
||||
this.pattern = pattern;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getPattern()
|
||||
{
|
||||
return pattern;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8);
|
||||
final byte[] patternBytes = pattern.getBytes(Charsets.UTF_8);
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length + patternBytes.length)
|
||||
.put(CACHE_ID_KEY)
|
||||
.put(dimensionBytes)
|
||||
.put(patternBytes)
|
||||
.array();
|
||||
}
|
||||
}
|
@ -0,0 +1,87 @@
|
||||
package com.metamx.druid.query.filter;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SelectorDimFilter implements DimFilter
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x0;
|
||||
|
||||
private final String dimension;
|
||||
private final String value;
|
||||
|
||||
@JsonCreator
|
||||
public SelectorDimFilter(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("value") String value
|
||||
)
|
||||
{
|
||||
this.dimension = dimension.toLowerCase();
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] dimensionBytes = dimension.getBytes();
|
||||
byte[] valueBytes = value.getBytes();
|
||||
|
||||
return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(dimensionBytes)
|
||||
.put(valueBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SelectorDimFilter that = (SelectorDimFilter) o;
|
||||
|
||||
if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) {
|
||||
return false;
|
||||
}
|
||||
if (value != null ? !value.equals(that.value) : that.value != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = dimension != null ? dimension.hashCode() : 0;
|
||||
result = 31 * result + (value != null ? value.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return String.format("%s = %s", dimension, value);
|
||||
}
|
||||
}
|
@ -0,0 +1,270 @@
|
||||
package com.metamx.druid.query.group;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.druid.BaseQuery;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.aggregation.post.PostAggregator;
|
||||
import com.metamx.druid.input.Row;
|
||||
import com.metamx.druid.query.dimension.DefaultDimensionSpec;
|
||||
import com.metamx.druid.query.dimension.DimensionSpec;
|
||||
import com.metamx.druid.query.filter.DimFilter;
|
||||
import com.metamx.druid.query.segment.LegacySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class GroupByQuery extends BaseQuery<Row>
|
||||
{
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private final DimFilter dimFilter;
|
||||
private final QueryGranularity granularity;
|
||||
private final List<DimensionSpec> dimensions;
|
||||
private final List<AggregatorFactory> aggregatorSpecs;
|
||||
private final List<PostAggregator> postAggregatorSpecs;
|
||||
|
||||
@JsonCreator
|
||||
public GroupByQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||
@JsonProperty("filter") DimFilter dimFilter,
|
||||
@JsonProperty("granularity") QueryGranularity granularity,
|
||||
@JsonProperty("dimensions") List<DimensionSpec> dimensions,
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregatorSpecs,
|
||||
@JsonProperty("postAggregations") List<PostAggregator> postAggregatorSpecs,
|
||||
@JsonProperty("context") Map<String, String> context
|
||||
)
|
||||
{
|
||||
super(dataSource, querySegmentSpec, context);
|
||||
this.dimFilter = dimFilter;
|
||||
this.granularity = granularity;
|
||||
this.dimensions = dimensions == null ? ImmutableList.<DimensionSpec>of() : dimensions;
|
||||
this.aggregatorSpecs = aggregatorSpecs;
|
||||
this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.<PostAggregator>of() : postAggregatorSpecs;
|
||||
|
||||
Preconditions.checkNotNull(this.granularity, "Must specify a granularity");
|
||||
Preconditions.checkNotNull(this.aggregatorSpecs, "Must specify at least one aggregator");
|
||||
}
|
||||
|
||||
@JsonProperty("filter")
|
||||
public DimFilter getDimFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public QueryGranularity getGranularity()
|
||||
{
|
||||
return granularity;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<DimensionSpec> getDimensions()
|
||||
{
|
||||
return dimensions;
|
||||
}
|
||||
|
||||
@JsonProperty("aggregations")
|
||||
public List<AggregatorFactory> getAggregatorSpecs()
|
||||
{
|
||||
return aggregatorSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty("postAggregations")
|
||||
public List<PostAggregator> getPostAggregatorSpecs()
|
||||
{
|
||||
return postAggregatorSpecs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasFilters()
|
||||
{
|
||||
return dimFilter != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "groupBy";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query withOverriddenContext(Map<String, String> contextOverride)
|
||||
{
|
||||
return new GroupByQuery(
|
||||
getDataSource(),
|
||||
getQuerySegmentSpec(),
|
||||
dimFilter,
|
||||
granularity,
|
||||
dimensions,
|
||||
aggregatorSpecs,
|
||||
postAggregatorSpecs,
|
||||
computeOverridenContext(contextOverride)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query withQuerySegmentSpec(QuerySegmentSpec spec)
|
||||
{
|
||||
return new GroupByQuery(
|
||||
getDataSource(),
|
||||
spec,
|
||||
dimFilter,
|
||||
granularity,
|
||||
dimensions,
|
||||
aggregatorSpecs,
|
||||
postAggregatorSpecs,
|
||||
getContext()
|
||||
);
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private String dataSource;
|
||||
private QuerySegmentSpec querySegmentSpec;
|
||||
private DimFilter dimFilter;
|
||||
private QueryGranularity granularity;
|
||||
private List<DimensionSpec> dimensions;
|
||||
private List<AggregatorFactory> aggregatorSpecs;
|
||||
private List<PostAggregator> postAggregatorSpecs;
|
||||
private Map<String, String> context;
|
||||
|
||||
private Builder() {}
|
||||
|
||||
private Builder(Builder builder)
|
||||
{
|
||||
dataSource = builder.dataSource;
|
||||
querySegmentSpec = builder.querySegmentSpec;
|
||||
dimFilter = builder.dimFilter;
|
||||
granularity = builder.granularity;
|
||||
dimensions = builder.dimensions;
|
||||
aggregatorSpecs = builder.aggregatorSpecs;
|
||||
postAggregatorSpecs = builder.postAggregatorSpecs;
|
||||
context = builder.context;
|
||||
}
|
||||
|
||||
public Builder setDataSource(String dataSource)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setInterval(Object interval)
|
||||
{
|
||||
return setQuerySegmentSpec(new LegacySegmentSpec(interval));
|
||||
}
|
||||
|
||||
public Builder setQuerySegmentSpec(QuerySegmentSpec querySegmentSpec)
|
||||
{
|
||||
this.querySegmentSpec = querySegmentSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setDimFilter(DimFilter dimFilter)
|
||||
{
|
||||
this.dimFilter = dimFilter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setGranularity(QueryGranularity granularity)
|
||||
{
|
||||
this.granularity = granularity;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder addDimension(String column)
|
||||
{
|
||||
return addDimension(column, column);
|
||||
}
|
||||
|
||||
public Builder addDimension(String column, String outputName)
|
||||
{
|
||||
return addDimension(new DefaultDimensionSpec(column, outputName));
|
||||
}
|
||||
|
||||
public Builder addDimension(DimensionSpec dimension)
|
||||
{
|
||||
if (dimensions == null) {
|
||||
dimensions = Lists.newArrayList();
|
||||
}
|
||||
|
||||
dimensions.add(dimension);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setDimensions(List<DimensionSpec> dimensions)
|
||||
{
|
||||
this.dimensions = Lists.newArrayList(dimensions);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder addAggregator(AggregatorFactory aggregator)
|
||||
{
|
||||
if (aggregatorSpecs == null) {
|
||||
aggregatorSpecs = Lists.newArrayList();
|
||||
}
|
||||
|
||||
aggregatorSpecs.add(aggregator);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setAggregatorSpecs(List<AggregatorFactory> aggregatorSpecs)
|
||||
{
|
||||
this.aggregatorSpecs = Lists.newArrayList(aggregatorSpecs);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder addPostAggregator(PostAggregator postAgg)
|
||||
{
|
||||
if (postAggregatorSpecs == null) {
|
||||
postAggregatorSpecs = Lists.newArrayList();
|
||||
}
|
||||
|
||||
postAggregatorSpecs.add(postAgg);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPostAggregatorSpecs(List<PostAggregator> postAggregatorSpecs)
|
||||
{
|
||||
this.postAggregatorSpecs = Lists.newArrayList(postAggregatorSpecs);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setContext(Map<String, String> context)
|
||||
{
|
||||
this.context = context;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder copy()
|
||||
{
|
||||
return new Builder(this);
|
||||
}
|
||||
|
||||
public GroupByQuery build()
|
||||
{
|
||||
return new GroupByQuery(
|
||||
dataSource,
|
||||
querySegmentSpec,
|
||||
dimFilter,
|
||||
granularity,
|
||||
dimensions,
|
||||
aggregatorSpecs,
|
||||
postAggregatorSpecs,
|
||||
context
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,161 @@
|
||||
package com.metamx.druid.query.group;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import com.metamx.common.guava.ConcatSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.index.v1.IncrementalIndex;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.input.Row;
|
||||
import com.metamx.druid.input.Rows;
|
||||
import com.metamx.druid.query.CacheStrategy;
|
||||
import com.metamx.druid.query.MetricManipulationFn;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChest;
|
||||
import com.metamx.druid.query.dimension.DimensionSpec;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQuery>
|
||||
{
|
||||
|
||||
private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>(){};
|
||||
|
||||
private static final int maxRows;
|
||||
|
||||
static {
|
||||
// I dislike this static loading of properies, but it's the only mechanism available right now.
|
||||
Properties props = Initialization.loadProperties();
|
||||
|
||||
maxRows = PropUtils.getPropertyAsInt(props, "com.metamx.query.groupBy.maxResults", 500000);
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Row> mergeResults(final QueryRunner<Row> runner)
|
||||
{
|
||||
return new QueryRunner<Row>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Row> run(Query<Row> input)
|
||||
{
|
||||
GroupByQuery query = (GroupByQuery) input;
|
||||
|
||||
List<Interval> condensed = query.getIntervals();
|
||||
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(
|
||||
condensed.get(0).getStartMillis(),
|
||||
query.getGranularity(),
|
||||
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;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return Sequences.simple(index);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<Row> mergeSequences(Sequence<Sequence<Row>> seqOfSequences)
|
||||
{
|
||||
return new ConcatSequence<Row>(seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServiceMetricEvent.Builder makeMetricBuilder(GroupByQuery query)
|
||||
{
|
||||
int numMinutes = 0;
|
||||
for (Interval interval : query.getIntervals()) {
|
||||
numMinutes += Minutes.minutesIn(interval).getMinutes();
|
||||
}
|
||||
|
||||
return new ServiceMetricEvent.Builder()
|
||||
.setUser2(query.getDataSource())
|
||||
.setUser3(String.format("%,d dims", query.getDimensions().size()))
|
||||
.setUser4("groupBy")
|
||||
.setUser5(Joiner.on(",").join(query.getIntervals()))
|
||||
.setUser6(String.valueOf(query.hasFilters()))
|
||||
.setUser7(String.format("%,d aggs", query.getAggregatorSpecs().size()))
|
||||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Row, Row> makeMetricManipulatorFn(GroupByQuery query, MetricManipulationFn fn)
|
||||
{
|
||||
return Functions.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Row> getResultTypeReference()
|
||||
{
|
||||
return TYPE_REFERENCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Row, GroupByQuery> getCacheStrategy(GroupByQuery query)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Row> preMergeQueryDecoration(QueryRunner<Row> runner)
|
||||
{
|
||||
return runner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Row> postMergeQueryDecoration(QueryRunner<Row> runner)
|
||||
{
|
||||
return runner;
|
||||
}
|
||||
}
|
@ -0,0 +1,94 @@
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FragmentSearchQuerySpec implements SearchQuerySpec
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x2;
|
||||
|
||||
private final List<String> values;
|
||||
private final SearchSortSpec sortSpec;
|
||||
|
||||
@JsonCreator
|
||||
public FragmentSearchQuerySpec(
|
||||
@JsonProperty("values") List<String> values,
|
||||
@JsonProperty("sort") SearchSortSpec sortSpec
|
||||
)
|
||||
{
|
||||
this.values = Lists.transform(
|
||||
values,
|
||||
new Function<String, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(String s)
|
||||
{
|
||||
return s.toLowerCase();
|
||||
}
|
||||
}
|
||||
);
|
||||
this.sortSpec = (sortSpec == null) ? new LexicographicSearchSortSpec() : sortSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<String> getValues()
|
||||
{
|
||||
return values;
|
||||
}
|
||||
|
||||
@JsonProperty("sort")
|
||||
@Override
|
||||
public SearchSortSpec getSearchSortSpec()
|
||||
{
|
||||
return sortSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(String dimVal)
|
||||
{
|
||||
for (String value : values) {
|
||||
if (!dimVal.toLowerCase().contains(value)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final byte[][] valuesBytes = new byte[values.size()][];
|
||||
int valuesBytesSize = 0;
|
||||
int index = 0;
|
||||
for (String value : values) {
|
||||
valuesBytes[index] = value.getBytes();
|
||||
valuesBytesSize += valuesBytes[index].length;
|
||||
++index;
|
||||
}
|
||||
|
||||
final ByteBuffer queryCacheKey = ByteBuffer.allocate(1 + valuesBytesSize)
|
||||
.put(CACHE_TYPE_ID);
|
||||
|
||||
for (byte[] bytes : valuesBytes) {
|
||||
queryCacheKey.put(bytes);
|
||||
}
|
||||
|
||||
return queryCacheKey.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "FragmentSearchQuerySpec{" +
|
||||
"values=" + values +
|
||||
", sortSpec=" + sortSpec +
|
||||
"}";
|
||||
}
|
||||
}
|
@ -0,0 +1,65 @@
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
|
||||
{
|
||||
private static final byte CACHE_TYPE_ID = 0x1;
|
||||
|
||||
private final String value;
|
||||
private final SearchSortSpec sortSpec;
|
||||
|
||||
@JsonCreator
|
||||
public InsensitiveContainsSearchQuerySpec(
|
||||
@JsonProperty("value") String value,
|
||||
@JsonProperty("sort") SearchSortSpec sortSpec
|
||||
)
|
||||
{
|
||||
this.value = value.toLowerCase();
|
||||
this.sortSpec = (sortSpec == null) ? new LexicographicSearchSortSpec() : sortSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@JsonProperty("sort")
|
||||
@Override
|
||||
public SearchSortSpec getSearchSortSpec()
|
||||
{
|
||||
return sortSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(String dimVal)
|
||||
{
|
||||
return dimVal.toLowerCase().contains(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] valueBytes = value.getBytes();
|
||||
|
||||
return ByteBuffer.allocate(1 + valueBytes.length)
|
||||
.put(CACHE_TYPE_ID)
|
||||
.put(valueBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "InsensitiveContainsSearchQuerySpec{" +
|
||||
"value=" + value +
|
||||
", sortSpec=" + sortSpec +
|
||||
"}";
|
||||
}
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LexicographicSearchSortSpec implements SearchSortSpec
|
||||
{
|
||||
@JsonCreator
|
||||
public LexicographicSearchSortSpec(
|
||||
)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<SearchHit> getComparator()
|
||||
{
|
||||
return new Comparator<SearchHit>()
|
||||
{
|
||||
@Override
|
||||
public int compare(SearchHit searchHit, SearchHit searchHit1)
|
||||
{
|
||||
return searchHit.getValue().compareTo(searchHit1.getValue());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public String toString()
|
||||
{
|
||||
return "lexicographicSort";
|
||||
}
|
||||
}
|
@ -0,0 +1,85 @@
|
||||
package com.metamx.druid.query.search;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SearchHit implements Comparable<SearchHit>
|
||||
{
|
||||
private final String dimension;
|
||||
private final String value;
|
||||
|
||||
@JsonCreator
|
||||
public SearchHit(
|
||||
@JsonProperty("dimension") String dimension,
|
||||
@JsonProperty("value") String value
|
||||
)
|
||||
{
|
||||
this.dimension = checkNotNull(dimension);
|
||||
this.value = checkNotNull(value);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
return dimension;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
{
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(SearchHit o)
|
||||
{
|
||||
int retVal = dimension.compareTo(o.dimension);
|
||||
if (retVal == 0) {
|
||||
retVal = value.compareTo(o.value);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SearchHit searchHit = (SearchHit) o;
|
||||
|
||||
if (dimension != null ? !dimension.equals(searchHit.dimension) : searchHit.dimension != null) {
|
||||
return false;
|
||||
}
|
||||
if (value != null ? !value.equals(searchHit.value) : searchHit.value != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = dimension != null ? dimension.hashCode() : 0;
|
||||
result = 31 * result + (value != null ? value.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "Hit{" +
|
||||
"dimension='" + dimension + '\'' +
|
||||
", value='" + value + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user